Skip to content

Conversation

@ajaygeorge
Copy link
Contributor

  • Prepare for next development iteration - 0.284-edge19-SNAPSHOT
  • Prepare for next development iteration - 0.284-edge19.1-SNAPSHOT
  • Prepare for next development iteration - 0.284-edge19.2-SNAPSHOT
  • Add split retry feature
  • Worker Isolation
  • Add Thrift serializer for LongSet
  • Fix worker isolation
  • Fix multiple issues with retry
  • Fix query runner
  • Fix stuck query on intermediate workers
  • Recovering opec worker
  • Schedule queries with LocalExchange to intermediate nodes
  • reduce no_more_retry_delay to 5 sec
  • setNoMorePages for shutting down task's outputbuffer
  • Handle FixedSourcePartitionedScheduler for splitting leaf and intermediate
  • Logging, metrics and handle race conditions
  • Handle race condition Add byte[] backed tuple and block implementations #2
  • Change the test to make the pageCount randomly generated
  • Add complex case for graceful shutdown
  • Fix the corner case where the outputBuffer has not received noMoreBuffer from the coordinator
  • Fix compilation errors
  • Use the latest retried logic
  • Add a new GRACEFUL_FAILED stsate for task
  • Attempt to fix TOO_MANY_REQUEST_FAILED in the verifier results
  • Using callback to decide whether to unblock the SqlStageExecution
  • Revert the cacnellation conditions
  • Add SplitRetrySourcePartitionedScheduler to support noMoreRetry
  • Commented out line for checking coordinator count
  • Add debug log
  • Fix noMoreSplits error on the intermediate stage

Description

Motivation and Context

Impact

Test Plan

Contributor checklist

  • Please make sure your submission complies with our development, formatting, commit message, and attribution guidelines.
  • PR description addresses the issue accurately and concisely. If the change is non-trivial, a GitHub Issue is referenced.
  • Documented new properties (with its default value), SQL syntax, functions, or other functionality.
  • If release notes are required, they follow the release notes guidelines.
  • Adequate tests were added if applicable.
  • CI passed.

Release Notes

Please follow release notes guidelines and fill in the release notes below.

== RELEASE NOTES ==

General Changes
* ...
* ...

Hive Changes
* ...
* ...

If release note is NOT required, use:

== NO RELEASE NOTE ==

presto-release-bot and others added 30 commits September 15, 2023 17:10
Add metric for retry and unit test for graceful shutdown
@ajaygeorge ajaygeorge changed the title ajay opec [WIP] ajay opec Oct 12, 2023
public void handleShutdown(TaskId taskId)
{
String errorMessage = String.format("killing pending task %s due to host being shutting down", taskId);
taskStateMachine.graceful_failed(new HostShuttingDownException(errorMessage, System.nanoTime()));
Copy link
Contributor Author

Choose a reason for hiding this comment

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

System.nanoTime is to measure elapsed time and not absolute time.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit : graceful_failed method name

taskStateMachine.cancel();
}

public void graceful_failed()
Copy link
Contributor Author

Choose a reason for hiding this comment

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

no one calls this

void start();

void addSplits(Multimap<PlanNodeId, Split> splits);
boolean addSplits(Multimap<PlanNodeId, Split> splits);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit : add javadoc to say what boolean return value means

return pendingSourceSplitCount;
}

public synchronized void setIsRetried()
Copy link
Contributor Author

Choose a reason for hiding this comment

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

why synchronized. ?


if (failedTasks.isEmpty()) {
checkState(finishedTasks.isEmpty());
List<RemoteTask> idleRunningHttpRemoteTasks = getAllTasks().stream()
Copy link
Contributor Author

@ajaygeorge ajaygeorge Oct 13, 2023

Choose a reason for hiding this comment

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

AGPQN idleRunningHttpRemoteTasks sounds like an oxymoron

Comment on lines +723 to +724
.filter(task -> task.getTaskStatus().getState() == TaskState.RUNNING)
.filter(task -> task.isTaskIdling())
Copy link
Contributor Author

Choose a reason for hiding this comment

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

combine both filters into one

private final AccessControlContext context;
private final Optional<Tracer> tracer;
private final WarningCollector warningCollector;
private final ConcurrentHashMap<PlanFragmentId, Pair<NodePoolType, String>> fragmentToPoolTypeMapping = new ConcurrentHashMap<>();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit. code to interfaces. ConcurrentMap

Comment on lines +547 to +554
ConcurrentHashMap<PlanFragmentId, Session.Pair<NodePoolType, String>> fragmentToPoolTypeMapping = session.getFragmentToPoolTypeMapping();
Session.Pair<NodePoolType, String> nodePoolTypeStringPair = fragmentToPoolTypeMapping.get(plan.getFragment().getId());
if (nodePoolTypeStringPair != null && nodePoolTypeStringPair.getKey() != workerPoolType) {
log.error("Error in pool type evaluation, plan =%s,hasLocalExchangeAtRoot = %s, isLeaf=%s, plan in map =%s", plan.getFragment().getJsonRepresentation().orElse(null), hasLocalExchangeAtRoot, plan.getFragment().isLeaf(), nodePoolTypeStringPair.getValue());
throw new RuntimeException("Error in pool type evaluation");
}
else {
fragmentToPoolTypeMapping.put(plan.getFragment().getId(), new Session.Pair<>(workerPoolType, plan.getFragment().getJsonRepresentation().orElse("")));
Copy link
Contributor Author

Choose a reason for hiding this comment

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

AGPQN when do we enter into the erorr situation.
Also why do we need the entire plan fragment json stuffed into the session

Comment on lines +666 to +669
if (taskSplit.getScheduledSplit() != null) {
log.warn("Adding split %s to pending split tracker for task %s", taskSplit.getScheduledSplit().getSequenceId(), taskHandle.getTaskId());
gracefulShutdownSplitTracker.getPendingSplits().computeIfAbsent(taskHandle.getTaskId(), k -> ConcurrentHashMap.newKeySet()).add(taskSplit.getScheduledSplit().getSequenceId());
}
Copy link
Contributor Author

@ajaygeorge ajaygeorge Oct 14, 2023

Choose a reason for hiding this comment

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

why is gracefulShutdownSplitTracker needed for non shutdown cases?

QUERY_PLANNING_TIMEOUT(0x0001_0027, INTERNAL_ERROR),
NATIVE_EXECUTION_TASK_ERROR(0x0001_0028, INTERNAL_ERROR),
UNSUPPORTED_ANALYZER_TYPE(0x0001_0029, INTERNAL_ERROR),
REMOTE_HOST_GONE_INTERMEDIATE(0x0001_0029, INTERNAL_ERROR, true),
Copy link
Contributor Author

Choose a reason for hiding this comment

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

not used anywhere

return tasks.getUnchecked(taskId).abort();
}

public void failTask(TaskId taskId)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

only called for testing

Comment on lines +332 to +338
{
for (ClientBuffer partition : partitions) {
if (!partition.isEmptyPages()) {
return false;
}
}
return true;
Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit this logic is duplicated in multiple output buffer implementations.

.withFilter(GenerateTraceTokenRequestFilter.class)
.withConfigDefaults(config -> {
config.setRequestTimeout(new Duration(10, SECONDS));
config.setRequestTimeout(new Duration(30, SECONDS));
Copy link
Contributor Author

Choose a reason for hiding this comment

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

random change?

AtomicReference<SqlQuerySchedulerInterface> queryScheduler = this.queryScheduler;
stateMachine.addStateChangeListener(state -> {
if (!state.isDone()) {
if (!state.isDone() || state == QueryState.FINISHED) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

AGPQN not Done or finished seems interesting.

Comment on lines +221 to +224
public void gracefulShutdown()
{
isShuttingDown.set(true);
}
Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit calling it graceful shutdown implies there is some action to be done.
maybe just call it as markTaskAsShutdown

private final TimeStat outputBufferEmptyWaitTime = new TimeStat(NANOSECONDS);
private final TimeStat waitForRunningSplitTime = new TimeStat(NANOSECONDS);
private volatile boolean closed;
private final ExecutorService taskShutdownExecutor = newCachedThreadPool(daemonThreadsNamed("task-shutdown-%s"));
Copy link
Contributor Author

@ajaygeorge ajaygeorge Oct 18, 2023

Choose a reason for hiding this comment

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

instead of a cached thread pool which starts from no threads , it might be better to go with a thread pool of min size = p90 number of tasks on a dying node since thread creation takes time.

builderWithOutputBufferInfo("init", shuttingdownNode, outputBuffer)
.build());

while (taskHandle.getRunningLeafSplits() > 0 || taskHandle.getRunningIntermediateSplits() > 0) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

AGPQN : why are we checking for taskHandle.getRunningIntermediateSplits()

Comment on lines +108 to +111
public static final String OB_WAIT_OVER = "obWait-over";
public static final String OB_WAIT = "obWait";
public static final String SPLIT_WAIT = "splitWait";
public static final String SPLIT_WAIT_OVER = "splitWait-over";
Copy link
Contributor Author

Choose a reason for hiding this comment

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

if we are planning to keep it in prod, might be good to add ordinals prefixed so that it is easier to understand the sequence of events

Comment on lines +281 to +283
catch (Exception ex) {
log.error(ex, "GracefulShutdown got interrupted while waiting for split completion for task %s", taskId);
}
Copy link
Contributor Author

Choose a reason for hiding this comment

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

If the intend is to catch Interrupted Exception, we need to specifically catch that and set the interrupt flag back.

{
isGracefulShutdownStarted.set(true);
long shutdownStartTime = System.nanoTime();
//TODO throw error for new task creation instead of looping here.
Copy link
Contributor Author

Choose a reason for hiding this comment

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

AGPQN : Are we creating tasks here?

Duration splitConcurrencyAdjustFrequency,
OptionalInt maxDriversPerTask)
{
return addTask(taskId, utilizationSupplier, initialSplitConcurrency, splitConcurrencyAdjustFrequency, maxDriversPerTask, Optional.empty(), Optional.empty());
Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit: get rid of this wrapper since the above method is only called in 1 place

Comment on lines +228 to +231
if (!hostShutDownListener.isPresent()) {
return;
}
hostShutDownListener.get().handleShutdown(taskId);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit : rewrite in functional style as

hostShutDownListener.ifPresent(taskShutDownListener -> taskShutDownListener.handleShutdown(taskId));

Comment on lines +276 to +279
if (!hostShutDownListener.isPresent()) {
return;
}
hostShutDownListener.get().addStats(taskShutdownStats);
Copy link
Contributor Author

@ajaygeorge ajaygeorge Oct 31, 2023

Choose a reason for hiding this comment

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

same as above

hostShutDownListener.ifPresent(taskShutDownListener -> taskShutDownListener.addStats(taskShutdownStats));

builder.addAll(runningLeafSplits);
builder.addAll(queuedLeafSplits);
//To avoid queued split marked as completed splits to pollute the retryable splits
if (isShuttingDown.get()) {
Copy link
Contributor Author

@ajaygeorge ajaygeorge Oct 31, 2023

Choose a reason for hiding this comment

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

AGPQN : This check is a bit confusing. Can we call this method destroy say if we want to do speculative execution and cancel/destroy the work which is independent of the notion of a shutdown.

queuedLeafSplits.add(split);
}
else {
checkState(!split.isSplitAlreadyStarted(), "Split we are avoiding to queue was already started!");
Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit avoiding -> adding

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants