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

[multistage] implement naive round robin operator chain scheduling #9753

Merged
merged 4 commits into from
Nov 15, 2022
Merged
Show file tree
Hide file tree
Changes from 3 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 @@ -44,7 +44,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
import org.testng.SkipException;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.DataProvider;
Expand All @@ -54,7 +53,6 @@

public class SSBQueryIntegrationTest extends BaseClusterIntegrationTest {
private static final Logger LOGGER = LoggerFactory.getLogger(SSBQueryIntegrationTest.class);
private static final int MIN_AVAILABLE_CORE_REQUIREMENT = 4;
private static final Map<String, String> SSB_QUICKSTART_TABLE_RESOURCES = ImmutableMap.of(
"customer", "examples/batch/ssb/customer",
"dates", "examples/batch/ssb/dates",
Expand All @@ -66,10 +64,6 @@ public class SSBQueryIntegrationTest extends BaseClusterIntegrationTest {
@BeforeClass
public void setUp()
throws Exception {
if (Runtime.getRuntime().availableProcessors() < MIN_AVAILABLE_CORE_REQUIREMENT) {
throw new SkipException("Skip SSB query testing. Insufficient core count: "
+ Runtime.getRuntime().availableProcessors());
}
Comment on lines -69 to -72
Copy link
Contributor

Choose a reason for hiding this comment

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

freaking awesome!!!

TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir);

// Start the Pinot cluster
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,11 +50,15 @@
import org.apache.pinot.query.mailbox.MultiplexingMailboxService;
import org.apache.pinot.query.planner.StageMetadata;
import org.apache.pinot.query.planner.stage.MailboxSendNode;
import org.apache.pinot.query.planner.stage.StageNode;
import org.apache.pinot.query.runtime.blocks.TransferableBlock;
import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
import org.apache.pinot.query.runtime.executor.WorkerQueryExecutor;
import org.apache.pinot.query.runtime.executor.OpChainSchedulerService;
import org.apache.pinot.query.runtime.operator.MailboxSendOperator;
import org.apache.pinot.query.runtime.operator.OpChain;
import org.apache.pinot.query.runtime.plan.DistributedStagePlan;
import org.apache.pinot.query.runtime.plan.PhysicalPlanVisitor;
import org.apache.pinot.query.runtime.plan.PlanRequestContext;
import org.apache.pinot.query.runtime.plan.ServerRequestPlanVisitor;
import org.apache.pinot.query.runtime.plan.server.ServerPlanRequestContext;
import org.apache.pinot.query.service.QueryConfig;
Expand All @@ -76,7 +80,6 @@ public class QueryRunner {
private static final Logger LOGGER = LoggerFactory.getLogger(QueryRunner.class);
// This is a temporary before merging the 2 type of executor.
private ServerQueryExecutorV1Impl _serverExecutor;
private WorkerQueryExecutor _workerExecutor;
private HelixManager _helixManager;
private ZkHelixPropertyStore<ZNRecord> _helixPropertyStore;
private MailboxService<TransferableBlock> _mailboxService;
Expand All @@ -98,8 +101,6 @@ public void init(PinotConfiguration config, InstanceDataManager instanceDataMana
_mailboxService = MultiplexingMailboxService.newInstance(_hostname, _port, config);
_serverExecutor = new ServerQueryExecutorV1Impl();
_serverExecutor.init(config, instanceDataManager, serverMetrics);
_workerExecutor = new WorkerQueryExecutor();
_workerExecutor.init(config, serverMetrics, _mailboxService, _hostname, _port);
} catch (Exception e) {
throw new RuntimeException(e);
}
Expand All @@ -109,16 +110,14 @@ public void start() {
_helixPropertyStore = _helixManager.getHelixPropertyStore();
_mailboxService.start();
_serverExecutor.start();
_workerExecutor.start();
}

public void shutDown() {
_workerExecutor.shutDown();
_serverExecutor.shutDown();
_mailboxService.shutdown();
}

public void processQuery(DistributedStagePlan distributedStagePlan, ExecutorService executorService,
public void processQuery(DistributedStagePlan distributedStagePlan, OpChainSchedulerService scheduler,
Map<String, String> requestMetadataMap) {
if (isLeafStage(distributedStagePlan)) {
// TODO: make server query request return via mailbox, this is a hack to gather the non-streaming data table
Expand All @@ -132,7 +131,7 @@ public void processQuery(DistributedStagePlan distributedStagePlan, ExecutorServ
for (ServerPlanRequestContext requestContext : serverQueryRequests) {
ServerQueryRequest request = new ServerQueryRequest(requestContext.getInstanceRequest(),
new ServerMetrics(PinotMetricUtils.getPinotMetricsRegistry()), System.currentTimeMillis());
serverQueryResults.add(processServerQuery(request, executorService));
serverQueryResults.add(processServerQuery(request, scheduler.getWorkerPool()));
Copy link
Contributor

Choose a reason for hiding this comment

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

this means leaf stage are directly scheduled on top of the worker pool?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, IIRC this is the same as existing behavior if you follow where executorService is created and passed down

}

MailboxSendNode sendNode = (MailboxSendNode) distributedStagePlan.getStageRoot();
Expand All @@ -148,7 +147,11 @@ public void processQuery(DistributedStagePlan distributedStagePlan, ExecutorServ
LOGGER.debug("Acquired transferable block: {}", blockCounter++);
}
} else {
_workerExecutor.processQuery(distributedStagePlan, requestMetadataMap, executorService);
long requestId = Long.parseLong(requestMetadataMap.get("REQUEST_ID"));
StageNode stageRoot = distributedStagePlan.getStageRoot();
OpChain rootOperator = PhysicalPlanVisitor.build(stageRoot, new PlanRequestContext(
_mailboxService, requestId, stageRoot.getStageId(), _hostname, _port, distributedStagePlan.getMetadataMap()));
scheduler.register(rootOperator);
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pinot.query.runtime.executor;

import org.apache.pinot.query.mailbox.MailboxIdentifier;
import org.apache.pinot.query.runtime.operator.OpChain;


/**
* An interface that defines different scheduling strategies to work with the
* {@link OpChainSchedulerService}. All methods are thread safe and can be guaranteed
* to never be called concurrently - therefore all implementations may use data
* structures that are not concurrent.
*/
public interface OpChainScheduler {

/**
* @param operatorChain the operator chain to register
*/
void register(OpChain operatorChain);

/**
* This method is called whenever {@code mailbox} has new data available to consume,
* this can be useful for advanced scheduling algorithms
*
* @param mailbox the mailbox ID
*/
void onDataAvailable(MailboxIdentifier mailbox);
Copy link
Contributor

Choose a reason for hiding this comment

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

is this the only trigger? (other than register)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

there's three possible triggers:

  • register
  • onDataAvailable
  • next().getRoot().nextBlock() completes

Triggers are defined in the implementation instead of the interface.


/**
* @return whether or not there is any work for the scheduler to do
*/
boolean hasNext();

/**
* @return the next operator chain to process
* @throws java.util.NoSuchElementException if {@link #hasNext()} returns false
* prior to this call
*/
OpChain next();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pinot.query.runtime.executor;

import com.google.common.util.concurrent.AbstractExecutionThreadService;
import com.google.common.util.concurrent.Monitor;
import java.util.concurrent.ExecutorService;
import org.apache.pinot.common.request.context.ThreadTimer;
import org.apache.pinot.core.util.trace.TraceRunnable;
import org.apache.pinot.query.mailbox.MailboxIdentifier;
import org.apache.pinot.query.runtime.blocks.TransferableBlock;
import org.apache.pinot.query.runtime.operator.OpChain;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


/**
* This class provides the implementation for scheduling multistage queries on a single node based
* on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
* and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
* chain ({@link OpChainScheduler#next()} will be requested.
*
* <p>Note that a yielded operator chain will be re-registered with the underlying scheduler.
*/
@SuppressWarnings("UnstableApiUsage")
public class OpChainSchedulerService extends AbstractExecutionThreadService {

private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);

private final OpChainScheduler _scheduler;
Copy link
Contributor

Choose a reason for hiding this comment

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

Add a comment saying this is guarded by monitor below?

private final ExecutorService _workerPool;

// anything that is guarded by this monitor should be non-blocking
private final Monitor _monitor = new Monitor();
protected final Monitor.Guard _hasNextOrClosing = new Monitor.Guard(_monitor) {
Copy link
Contributor

Choose a reason for hiding this comment

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

this is more of a question --> why one of these is private and the other is protected?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

that is a very good question... I think it was an autocomplete typo

@Override
public boolean isSatisfied() {
return _scheduler.hasNext() || !isRunning();
}
};

public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
_scheduler = scheduler;
_workerPool = workerPool;
}

@Override
protected void triggerShutdown() {
// this wil just notify all waiters that the scheduler is shutting down
_monitor.enter();
_monitor.leave();
}

@Override
protected void run()
throws Exception {
while (isRunning()) {
_monitor.enterWhen(_hasNextOrClosing);
try {
if (!isRunning()) {
return;
}

OpChain operatorChain = _scheduler.next();
_workerPool.submit(new TraceRunnable() {
@Override
public void runJob() {
try {
ThreadTimer timer = operatorChain.getAndStartTimer();

// so long as there's work to be done, keep getting the next block
// when the operator chain returns a NOOP block, then yield the execution
// of this to another worker
TransferableBlock result = operatorChain.getRoot().nextBlock();
Copy link
Contributor

Choose a reason for hiding this comment

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

IIUC, for the current mechanism, this will return NO-OP everytime it reaches all the way down to mailbox receive and the buffer is empty. yes?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

👍 yup that's correct

while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
LOGGER.debug("Got block with {} rows.", result.getNumRows());
result = operatorChain.getRoot().nextBlock();
}

if (!result.isEndOfStreamBlock()) {
// not complete, needs to re-register for scheduling
register(operatorChain);
} else {
LOGGER.info("Execution time: " + timer.getThreadTimeNs());
Copy link
Contributor Author

Choose a reason for hiding this comment

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

for more complex scheduling algorithms, we will add a callback here to complete or unregister an operator chain. that requires a unique way to identify operator chains which adds a bit more code so I avoided it for this PR

Copy link
Contributor

Choose a reason for hiding this comment

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

+1. good splitting point

Copy link
Contributor

Choose a reason for hiding this comment

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

Is this logging expensive? I feel there would be a lot of logs if we log the pause every time. Can we have some class such as OpChainStats to hold the data and we decide later where to report them?

Copy link
Contributor

Choose a reason for hiding this comment

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

oops good callout. this could be a debug log

}
} catch (Exception e) {
LOGGER.error("Failed to execute query!", e);
}
Comment on lines +102 to +104
Copy link
Contributor

Choose a reason for hiding this comment

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

i dont think this is returnable via data blocks. echo back to @61yao's comment on timeout. maybe we need other ways to indicate out-of-norm failures. let's follow up on other PR (i don't think we handle these correctly right now either)

}
});
} finally {
_monitor.leave();
}
}
}

/**
* Register a new operator chain with the scheduler.
*
* @param operatorChain the chain to register
*/
public final void register(OpChain operatorChain) {
_monitor.enter();
try {
_scheduler.register(operatorChain);
} finally {
_monitor.leave();
}
}

/**
* This method should be called whenever data is available in a given mailbox.
* Implementations of this method should be idempotent, it may be called in the
* scenario that no mail is available.
*
* @param mailbox the identifier of the mailbox that now has data
*/
public final void onDataAvailable(MailboxIdentifier mailbox) {
_monitor.enter();
try {
_scheduler.onDataAvailable(mailbox);
} finally {
_monitor.leave();
}
}

// TODO: remove this method after we pipe down the proper executor pool to the v1 engine
Copy link
Contributor Author

Choose a reason for hiding this comment

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

this is a bit unfortunate, but it's how the existing code works and refactoring it would be out of scope for this PR. While it's not particularly efficient, it also isn't dangerous - V1 queries are non-blocking, so using the same worker pool for executing V1 queries (that are issued as part of V2) and V2 intermediate queries does not threaten liveness.

Copy link
Contributor

Choose a reason for hiding this comment

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

lol should've read this first. good call out.

another way is to simple decoupled the executor service used by v1 from v2. i am not sure which is better.

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 think we should split v1/v2 executor pools - that's probably the safest option. Alternatively we may also want three pools: a v2-intermediate pool, a v1-via-v2 pool and a v1-vanilla pool. That will allow us to make sure that clusters that run an existing v1 vanilla workload have their exposure to the v2 engine entirely limited

Copy link
Contributor

Choose a reason for hiding this comment

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

+1, but for another day

public ExecutorService getWorkerPool() {
return _workerPool;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pinot.query.runtime.executor;

import java.util.LinkedList;
import java.util.Queue;
import org.apache.pinot.query.mailbox.MailboxIdentifier;
import org.apache.pinot.query.runtime.operator.OpChain;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


public class RoundRobinScheduler implements OpChainScheduler {
private static final Logger LOGGER = LoggerFactory.getLogger(RoundRobinScheduler.class);

private final Queue<OpChain> _opChainQueue = new LinkedList<>();

@Override
public void register(OpChain operatorChain) {
_opChainQueue.add(operatorChain);
}

@Override
public void onDataAvailable(MailboxIdentifier mailbox) {
// do nothing - this doesn't change order of execution
}

@Override
public boolean hasNext() {
// don't use _nextOpChain.hasNext() because that may potentially create
// a new iterator that gets tossed
return !_opChainQueue.isEmpty();
}

@Override
public OpChain next() {
return _opChainQueue.poll();
}
}
Loading