Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Fix leak of failed/aborted queries in coordinator
- Loading branch information
Showing
2 changed files
with
254 additions
and
63 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
179 changes: 179 additions & 0 deletions
179
presto-main/src/test/java/com/facebook/presto/execution/TestSqlStageExecution.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,179 @@ | ||
/* | ||
* Licensed 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 com.facebook.presto.execution; | ||
|
||
import com.facebook.presto.client.NodeVersion; | ||
import com.facebook.presto.cost.StatsAndCosts; | ||
import com.facebook.presto.execution.TestSqlTaskManager.MockLocationFactory; | ||
import com.facebook.presto.execution.scheduler.SplitSchedulerStats; | ||
import com.facebook.presto.failureDetector.NoOpFailureDetector; | ||
import com.facebook.presto.metadata.PrestoNode; | ||
import com.facebook.presto.operator.StageExecutionStrategy; | ||
import com.facebook.presto.spi.Node; | ||
import com.facebook.presto.spi.QueryId; | ||
import com.facebook.presto.spi.type.Type; | ||
import com.facebook.presto.sql.planner.Partitioning; | ||
import com.facebook.presto.sql.planner.PartitioningScheme; | ||
import com.facebook.presto.sql.planner.PlanFragment; | ||
import com.facebook.presto.sql.planner.Symbol; | ||
import com.facebook.presto.sql.planner.plan.PlanFragmentId; | ||
import com.facebook.presto.sql.planner.plan.PlanNode; | ||
import com.facebook.presto.sql.planner.plan.PlanNodeId; | ||
import com.facebook.presto.sql.planner.plan.RemoteSourceNode; | ||
import com.facebook.presto.util.FinalizerService; | ||
import com.google.common.collect.ImmutableList; | ||
import com.google.common.collect.ImmutableMap; | ||
import com.google.common.util.concurrent.SettableFuture; | ||
import org.testng.annotations.AfterClass; | ||
import org.testng.annotations.BeforeClass; | ||
import org.testng.annotations.Test; | ||
|
||
import java.net.URI; | ||
import java.util.Optional; | ||
import java.util.OptionalInt; | ||
import java.util.concurrent.CountDownLatch; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
|
||
import static com.facebook.presto.OutputBuffers.BufferType.ARBITRARY; | ||
import static com.facebook.presto.OutputBuffers.createInitialEmptyOutputBuffers; | ||
import static com.facebook.presto.SessionTestUtils.TEST_SESSION; | ||
import static com.facebook.presto.spi.type.VarcharType.VARCHAR; | ||
import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; | ||
import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; | ||
import static com.facebook.presto.sql.planner.plan.ExchangeNode.Type.REPARTITION; | ||
import static io.airlift.concurrent.Threads.daemonThreadsNamed; | ||
import static java.util.concurrent.Executors.newCachedThreadPool; | ||
import static java.util.concurrent.Executors.newScheduledThreadPool; | ||
import static java.util.concurrent.TimeUnit.MINUTES; | ||
import static org.testng.Assert.assertFalse; | ||
import static org.testng.Assert.assertTrue; | ||
|
||
public class TestSqlStageExecution | ||
{ | ||
private ExecutorService executor; | ||
private ScheduledExecutorService scheduledExecutor; | ||
|
||
@BeforeClass | ||
public void setUp() | ||
{ | ||
executor = newCachedThreadPool(daemonThreadsNamed("test-executor-%s")); | ||
scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed("test-scheduledExecutor-%s")); | ||
} | ||
|
||
@AfterClass | ||
public void tearDown() | ||
{ | ||
executor.shutdownNow(); | ||
scheduledExecutor.shutdownNow(); | ||
} | ||
|
||
@Test(timeOut = 2 * 60 * 1000) | ||
public void testFinalStageInfo() | ||
throws Exception | ||
{ | ||
// run test a few times to catch any race conditions | ||
// this is not done with TestNG invocation count so there can be a global time limit on the test | ||
for (int iteration = 0; iteration < 10; iteration++) { | ||
testFinalStageInfoInternal(); | ||
} | ||
} | ||
|
||
private void testFinalStageInfoInternal() | ||
throws Exception | ||
{ | ||
NodeTaskMap nodeTaskMap = new NodeTaskMap(new FinalizerService()); | ||
|
||
StageId stageId = new StageId(new QueryId("query"), 0); | ||
SqlStageExecution stage = new SqlStageExecution( | ||
stageId, | ||
new MockLocationFactory().createStageLocation(stageId), | ||
createExchangePlanFragment(), | ||
new MockRemoteTaskFactory(executor, scheduledExecutor), | ||
TEST_SESSION, | ||
true, | ||
nodeTaskMap, | ||
executor, | ||
new NoOpFailureDetector(), | ||
new SplitSchedulerStats()); | ||
stage.setOutputBuffers(createInitialEmptyOutputBuffers(ARBITRARY)); | ||
|
||
// add listener that fetches stage info when the final status is available | ||
SettableFuture<StageInfo> finalStageInfo = SettableFuture.create(); | ||
stage.addFinalStatusListener(value -> finalStageInfo.set(stage.getStageInfo())); | ||
|
||
// in a background thread add a ton of tasks | ||
CountDownLatch latch = new CountDownLatch(1000); | ||
Future<?> addTasksTask = executor.submit(() -> { | ||
try { | ||
for (int i = 0; i < 1_000_000; i++) { | ||
if (Thread.interrupted()) { | ||
return; | ||
} | ||
Node node = new PrestoNode( | ||
"source" + i, | ||
URI.create("http://10.0.0." + (i / 10_000) + ":" + (i % 10_000)), | ||
NodeVersion.UNKNOWN, | ||
false); | ||
stage.scheduleTask(node, i, OptionalInt.empty()); | ||
latch.countDown(); | ||
} | ||
} | ||
finally { | ||
while (latch.getCount() > 0) { | ||
latch.countDown(); | ||
} | ||
} | ||
}); | ||
|
||
// wait for some tasks to be created, and then abort the query | ||
latch.await(1, MINUTES); | ||
assertFalse(stage.getStageInfo().getTasks().isEmpty()); | ||
stage.abort(); | ||
|
||
// once the final stage info is available, verify that it is complete | ||
StageInfo stageInfo = finalStageInfo.get(1, MINUTES); | ||
assertFalse(stageInfo.getTasks().isEmpty()); | ||
assertTrue(stageInfo.isCompleteInfo()); | ||
assertTrue(stage.getStageInfo().isCompleteInfo()); | ||
|
||
// cancel the background thread adding tasks | ||
addTasksTask.cancel(true); | ||
} | ||
|
||
private static PlanFragment createExchangePlanFragment() | ||
{ | ||
PlanNode planNode = new RemoteSourceNode( | ||
new PlanNodeId("exchange"), | ||
ImmutableList.of(new PlanFragmentId("source")), | ||
ImmutableList.of(new Symbol("column")), | ||
Optional.empty(), | ||
REPARTITION); | ||
|
||
ImmutableMap.Builder<Symbol, Type> types = ImmutableMap.builder(); | ||
for (Symbol symbol : planNode.getOutputSymbols()) { | ||
types.put(symbol, VARCHAR); | ||
} | ||
return new PlanFragment( | ||
new PlanFragmentId("exchange_fragment_id"), | ||
planNode, | ||
types.build(), | ||
SOURCE_DISTRIBUTION, | ||
ImmutableList.of(planNode.getId()), | ||
new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), planNode.getOutputSymbols()), | ||
StageExecutionStrategy.ungroupedExecution(), | ||
StatsAndCosts.empty()); | ||
} | ||
} |