public static SqlTaskExecution createSqlTaskExecution( TaskStateMachine taskStateMachine, TaskContext taskContext, SharedBuffer sharedBuffer, PlanFragment fragment, List<TaskSource> sources, LocalExecutionPlanner planner, TaskExecutor taskExecutor, Executor notificationExecutor, QueryMonitor queryMonitor) { SqlTaskExecution task = new SqlTaskExecution( taskStateMachine, taskContext, sharedBuffer, fragment, planner, taskExecutor, queryMonitor, notificationExecutor); try (SetThreadName ignored = new SetThreadName("Task-%s", task.getTaskId())) { task.start(); task.addSources(sources); return task; } }
@Override public void stateChanged(BufferState taskState) { if (taskState == BufferState.FINISHED) { SqlTaskExecution sqlTaskExecution = sqlTaskExecutionReference.get(); if (sqlTaskExecution != null) { sqlTaskExecution.checkTaskCompletion(); } } }
@Test public void testBufferNotCloseOnFail() throws Exception { outputBuffers = outputBuffers .withBuffer("out", new UnpartitionedPagePartitionFunction()) .withNoMoreBufferIds(); taskExecution.addResultQueue(outputBuffers); BufferResult bufferResult = taskExecution.getResults( "out", 0, new DataSize(1, Unit.MEGABYTE), new Duration(0, TimeUnit.MILLISECONDS)); assertFalse(bufferResult.isBufferClosed()); bufferResult = taskExecution.getResults( "out", 0, new DataSize(1, Unit.MEGABYTE), new Duration(0, TimeUnit.MILLISECONDS)); assertFalse(bufferResult.isBufferClosed()); taskExecution.fail(new Exception("test")); assertEquals(taskExecution.getTaskInfo(false).getState(), TaskState.FAILED); // buffer will not be closed by fail event. event is async so wait for 500 MS for event to fire bufferResult = taskExecution.getResults( "out", 0, new DataSize(1, Unit.MEGABYTE), new Duration(500, TimeUnit.MILLISECONDS)); assertFalse(bufferResult.isBufferClosed()); bufferResult = taskExecution.getResults( "out", 0, new DataSize(1, Unit.MEGABYTE), new Duration(500, TimeUnit.MILLISECONDS)); assertFalse(bufferResult.isBufferClosed()); }
@Test public void testBufferCloseOnFinish() throws Exception { outputBuffers = outputBuffers .withBuffer("out", new UnpartitionedPagePartitionFunction()) .withNoMoreBufferIds(); taskExecution.addResultQueue(outputBuffers); BufferResult bufferResult = taskExecution.getResults( "out", 0, new DataSize(1, Unit.MEGABYTE), new Duration(0, TimeUnit.MILLISECONDS)); assertFalse(bufferResult.isBufferClosed()); bufferResult = taskExecution.getResults( "out", 0, new DataSize(1, Unit.MEGABYTE), new Duration(0, TimeUnit.MILLISECONDS)); assertFalse(bufferResult.isBufferClosed()); taskExecution.addSources( ImmutableList.of(new TaskSource(tableScanNodeId, ImmutableSet.<ScheduledSplit>of(), true))); assertEquals(taskExecution.getTaskInfo(false).getState(), TaskState.FINISHED); // buffer will be closed by cancel event (wait for 500 MS for event to fire) bufferResult = taskExecution.getResults( "out", 0, new DataSize(1, Unit.MEGABYTE), new Duration(500, TimeUnit.MILLISECONDS)); assertTrue(bufferResult.isBufferClosed()); bufferResult = taskExecution.getResults( "out", 0, new DataSize(1, Unit.MEGABYTE), new Duration(500, TimeUnit.MILLISECONDS)); assertTrue(bufferResult.isBufferClosed()); }
public static SqlTaskExecution createSqlTaskExecution( Session session, TaskId taskId, URI location, PlanFragment fragment, List<TaskSource> sources, OutputBuffers outputBuffers, LocalExecutionPlanner planner, DataSize maxBufferSize, TaskExecutor taskExecutor, ExecutorService notificationExecutor, DataSize maxTaskMemoryUsage, DataSize operatorPreAllocatedMemory, QueryMonitor queryMonitor, boolean cpuTimerEnabled) { SqlTaskExecution task = new SqlTaskExecution( session, taskId, location, fragment, outputBuffers, planner, maxBufferSize, taskExecutor, maxTaskMemoryUsage, operatorPreAllocatedMemory, queryMonitor, notificationExecutor, cpuTimerEnabled); try (SetThreadName setThreadName = new SetThreadName("Task-%s", taskId)) { task.start(); task.addSources(sources); task.recordHeartbeat(); return task; } }
@BeforeMethod public void setUp() throws Exception { DualMetadata dualMetadata = new DualMetadata(); TableHandle tableHandle = dualMetadata.getTableHandle(new SchemaTableName("default", DualMetadata.NAME)); assertNotNull(tableHandle, "tableHandle is null"); ColumnHandle columnHandle = dualMetadata.getColumnHandle(tableHandle, DualMetadata.COLUMN_NAME); assertNotNull(columnHandle, "columnHandle is null"); Symbol symbol = new Symbol(DualMetadata.COLUMN_NAME); MetadataManager metadata = new MetadataManager(new FeaturesConfig()); metadata.addInternalSchemaMetadata(MetadataManager.INTERNAL_CONNECTOR_ID, dualMetadata); DualSplitManager dualSplitManager = new DualSplitManager(new InMemoryNodeManager()); PartitionResult partitionResult = dualSplitManager.getPartitions(tableHandle, TupleDomain.all()); SplitSource splitSource = dualSplitManager.getPartitionSplits(tableHandle, partitionResult.getPartitions()); split = Iterables.getOnlyElement(splitSource.getNextBatch(1)); assertTrue(splitSource.isFinished()); LocalExecutionPlanner planner = new LocalExecutionPlanner( new NodeInfo("test"), metadata, new DataStreamManager(new DualDataStreamProvider()), new MockLocalStorageManager(new File("target/temp")), new RecordSinkManager(), new MockExchangeClientSupplier(), new ExpressionCompiler(metadata)); taskExecutor = new TaskExecutor(8); taskExecutor.start(); tableScanNodeId = new PlanNodeId("tableScan"); PlanFragment testFragment = new PlanFragment( new PlanFragmentId("fragment"), new TableScanNode( tableScanNodeId, tableHandle, ImmutableList.of(symbol), ImmutableMap.of(symbol, columnHandle), null, Optional.<GeneratedPartitions>absent()), ImmutableMap.of(symbol, Type.VARCHAR), PlanDistribution.SOURCE, tableScanNodeId, OutputPartitioning.NONE, ImmutableList.<Symbol>of()); TaskId taskId = new TaskId("query", "stage", "task"); Session session = new Session("user", "test", "default", "default", "test", "test"); taskNotificationExecutor = Executors.newCachedThreadPool(threadsNamed("task-notification-%d")); outputBuffers = OutputBuffers.INITIAL_EMPTY_OUTPUT_BUFFERS; taskExecution = SqlTaskExecution.createSqlTaskExecution( session, taskId, URI.create("fake://task/" + taskId), testFragment, ImmutableList.<TaskSource>of(), outputBuffers, planner, new DataSize(32, Unit.MEGABYTE), taskExecutor, taskNotificationExecutor, new DataSize(256, Unit.MEGABYTE), new DataSize(8, Unit.MEGABYTE), new QueryMonitor( new ObjectMapperProvider().get(), new NullEventClient(), new NodeInfo("test")), false); }