@Test public void testBufferCloseOnFinish() throws Exception { SqlTask sqlTask = createInitialTask(); OutputBuffers outputBuffers = INITIAL_EMPTY_OUTPUT_BUFFERS .withBuffer("out", new UnpartitionedPagePartitionFunction()) .withNoMoreBufferIds(); updateTask(sqlTask, EMPTY_SOURCES, outputBuffers); ListenableFuture<BufferResult> bufferResult = sqlTask.getTaskResults("out", 0, new DataSize(1, MEGABYTE)); assertFalse(bufferResult.isDone()); // finish the task by closing the sources (no splits will ever be added) updateTask( sqlTask, ImmutableList.of( new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.<ScheduledSplit>of(), true)), outputBuffers); assertEquals(sqlTask.getTaskInfo().getState(), TaskState.FINISHED); // buffer will be closed by cancel event (wait for 500 MS for event to fire) assertTrue(bufferResult.get(200, MILLISECONDS).isBufferClosed()); assertEquals(sqlTask.getTaskInfo().getOutputBuffers().getState(), BufferState.FINISHED); // verify the buffer is closed bufferResult = sqlTask.getTaskResults("out", 0, new DataSize(1, MEGABYTE)); assertTrue(bufferResult.isDone()); assertTrue(bufferResult.get().isBufferClosed()); }
@Test public void testBufferNotCloseOnFail() throws Exception { SqlTask sqlTask = createInitialTask(); updateTask( sqlTask, EMPTY_SOURCES, INITIAL_EMPTY_OUTPUT_BUFFERS.withBuffer("out", new UnpartitionedPagePartitionFunction())); ListenableFuture<BufferResult> bufferResult = sqlTask.getTaskResults("out", 0, new DataSize(1, MEGABYTE)); assertFalse(bufferResult.isDone()); TaskState taskState = sqlTask.getTaskInfo().getState(); sqlTask.failed(new Exception("test")); assertEquals( sqlTask.getTaskInfo(taskState).get(200, MILLISECONDS).getState(), TaskState.FAILED); // buffer will not be closed by fail event. event is async so wait a bit for event to fire try { assertTrue(bufferResult.get(200, MILLISECONDS).isBufferClosed()); fail("expected TimeoutException"); } catch (TimeoutException expected) { } assertFalse(sqlTask.getTaskResults("out", 0, new DataSize(1, MEGABYTE)).isDone()); }
@Test public void testAbort() throws Exception { SqlTask sqlTask = createInitialTask(); TaskInfo taskInfo = sqlTask.updateTask( SESSION, PLAN_FRAGMENT, ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(SPLIT), true)), INITIAL_EMPTY_OUTPUT_BUFFERS .withBuffer("out", new UnpartitionedPagePartitionFunction()) .withNoMoreBufferIds()); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.RUNNING); sqlTask.abortTaskResults("out"); taskInfo = sqlTask.getTaskInfo(taskInfo.getState()).get(1, TimeUnit.SECONDS); assertEquals(taskInfo.getState(), TaskState.FINISHED); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.FINISHED); }
@Test public void testCancel() throws Exception { SqlTask sqlTask = createInitialTask(); TaskInfo taskInfo = sqlTask.updateTask( SESSION, PLAN_FRAGMENT, ImmutableList.<TaskSource>of(), INITIAL_EMPTY_OUTPUT_BUFFERS); assertEquals(taskInfo.getState(), TaskState.RUNNING); assertNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.RUNNING); assertNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTask.cancel(); assertEquals(taskInfo.getState(), TaskState.CANCELED); assertNotNull(taskInfo.getStats().getEndTime()); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.CANCELED); assertNotNull(taskInfo.getStats().getEndTime()); }
@Test public void testEmptyQuery() throws Exception { SqlTask sqlTask = createInitialTask(); TaskInfo taskInfo = sqlTask.updateTask( SESSION, PLAN_FRAGMENT, ImmutableList.<TaskSource>of(), INITIAL_EMPTY_OUTPUT_BUFFERS); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTask.updateTask( SESSION, PLAN_FRAGMENT, ImmutableList.of( new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.<ScheduledSplit>of(), true)), INITIAL_EMPTY_OUTPUT_BUFFERS.withNoMoreBufferIds()); assertEquals(taskInfo.getState(), TaskState.FINISHED); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.FINISHED); }
@Test public void testBufferCloseOnCancel() throws Exception { SqlTask sqlTask = createInitialTask(); updateTask( sqlTask, EMPTY_SOURCES, INITIAL_EMPTY_OUTPUT_BUFFERS.withBuffer("out", new UnpartitionedPagePartitionFunction())); ListenableFuture<BufferResult> bufferResult = sqlTask.getTaskResults("out", 0, new DataSize(1, MEGABYTE)); assertFalse(bufferResult.isDone()); sqlTask.cancel(); assertEquals(sqlTask.getTaskInfo().getState(), TaskState.CANCELED); // buffer will be closed by cancel event.. the event is async so wait a bit for event to // propagate assertTrue(bufferResult.get(200, MILLISECONDS).isBufferClosed()); bufferResult = sqlTask.getTaskResults("out", 0, new DataSize(1, MEGABYTE)); assertTrue(bufferResult.isDone()); assertTrue(bufferResult.get().isBufferClosed()); }
@Test public void testSimpleQuery() throws Exception { SqlTask sqlTask = createInitialTask(); TaskInfo taskInfo = sqlTask.updateTask( SESSION, PLAN_FRAGMENT, ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(SPLIT), true)), INITIAL_EMPTY_OUTPUT_BUFFERS .withBuffer("out", new UnpartitionedPagePartitionFunction()) .withNoMoreBufferIds()); assertEquals(taskInfo.getState(), TaskState.RUNNING); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.RUNNING); BufferResult results = sqlTask.getTaskResults("out", 0, new DataSize(1, Unit.MEGABYTE)).get(); assertEquals(results.isBufferClosed(), false); assertEquals(results.getPages().size(), 1); assertEquals(results.getPages().get(0).getPositionCount(), 1); results = sqlTask .getTaskResults( "out", results.getToken() + results.getPages().size(), new DataSize(1, Unit.MEGABYTE)) .get(); assertEquals(results.isBufferClosed(), true); assertEquals(results.getPages().size(), 0); taskInfo = sqlTask.getTaskInfo(taskInfo.getState()).get(1, TimeUnit.SECONDS); assertEquals(taskInfo.getState(), TaskState.FINISHED); taskInfo = sqlTask.getTaskInfo(); assertEquals(taskInfo.getState(), TaskState.FINISHED); }