mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-3290: fix transient test failures in WorkerSourceTaskTest
Author: Jason Gustafson <jason@confluent.io> Reviewers: Gwen Shapira Closes #998 from hachikuji/KAFKA-3290
This commit is contained in:
		
							parent
							
								
									f676cfeb83
								
							
						
					
					
						commit
						cfc324333f
					
				|  | @ -145,6 +145,9 @@ abstract class WorkerTask implements Runnable { | |||
|         } catch (Throwable t) { | ||||
|             if (!cancelled.get()) | ||||
|                 lifecycleListener.onFailure(id, t); | ||||
| 
 | ||||
|             if (t instanceof Error) | ||||
|                 throw t; | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -17,11 +17,11 @@ | |||
| 
 | ||||
| package org.apache.kafka.connect.runtime; | ||||
| 
 | ||||
| import org.apache.kafka.common.utils.SystemTime; | ||||
| import org.apache.kafka.clients.producer.KafkaProducer; | ||||
| import org.apache.kafka.clients.producer.ProducerRecord; | ||||
| import org.apache.kafka.clients.producer.RecordMetadata; | ||||
| import org.apache.kafka.common.TopicPartition; | ||||
| import org.apache.kafka.common.utils.SystemTime; | ||||
| import org.apache.kafka.common.utils.Utils; | ||||
| import org.apache.kafka.connect.data.Schema; | ||||
| import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; | ||||
|  | @ -60,9 +60,8 @@ import java.util.concurrent.TimeUnit; | |||
| import java.util.concurrent.TimeoutException; | ||||
| 
 | ||||
| import static org.junit.Assert.assertEquals; | ||||
| import static org.junit.Assert.assertFalse; | ||||
| import static org.junit.Assert.assertTrue; | ||||
| import static org.junit.Assert.assertNull; | ||||
| import static org.junit.Assert.assertTrue; | ||||
| 
 | ||||
| @RunWith(PowerMockRunner.class) | ||||
| public class WorkerSourceTaskTest extends ThreadedTest { | ||||
|  | @ -200,8 +199,6 @@ public class WorkerSourceTaskTest extends ThreadedTest { | |||
|         final CountDownLatch pollLatch = expectPolls(1); | ||||
|         expectOffsetFlush(true); | ||||
| 
 | ||||
|         sourceTask.commit(); | ||||
|         EasyMock.expectLastCall(); | ||||
|         sourceTask.stop(); | ||||
|         EasyMock.expectLastCall(); | ||||
|         expectOffsetFlush(true); | ||||
|  | @ -235,11 +232,11 @@ public class WorkerSourceTaskTest extends ThreadedTest { | |||
| 
 | ||||
|         // We'll wait for some data, then trigger a flush | ||||
|         final CountDownLatch pollLatch = expectPolls(1); | ||||
|         expectOffsetFlush(false); | ||||
|         expectOffsetFlush(true); | ||||
| 
 | ||||
|         sourceTask.stop(); | ||||
|         EasyMock.expectLastCall(); | ||||
|         expectOffsetFlush(true); | ||||
|         expectOffsetFlush(false); | ||||
| 
 | ||||
|         statusListener.onShutdown(taskId); | ||||
|         EasyMock.expectLastCall(); | ||||
|  | @ -249,7 +246,7 @@ public class WorkerSourceTaskTest extends ThreadedTest { | |||
|         workerTask.initialize(EMPTY_TASK_PROPS); | ||||
|         executor.submit(workerTask); | ||||
|         awaitPolls(pollLatch); | ||||
|         assertFalse(workerTask.commitOffsets()); | ||||
|         assertTrue(workerTask.commitOffsets()); | ||||
|         workerTask.stop(); | ||||
|         assertEquals(true, workerTask.awaitStop(1000)); | ||||
| 
 | ||||
|  | @ -319,9 +316,9 @@ public class WorkerSourceTaskTest extends ThreadedTest { | |||
|         sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); | ||||
|         EasyMock.expectLastCall(); | ||||
|         sourceTask.start(EMPTY_TASK_PROPS); | ||||
|         statusListener.onStartup(taskId); | ||||
|         EasyMock.expectLastCall(); | ||||
| 
 | ||||
|         statusListener.onStartup(taskId); | ||||
|         EasyMock.expectLastCall().andAnswer(new IAnswer<Object>() { | ||||
|             @Override | ||||
|             public Object answer() throws Throwable { | ||||
|  | @ -330,8 +327,10 @@ public class WorkerSourceTaskTest extends ThreadedTest { | |||
|                 return null; | ||||
|             } | ||||
|         }); | ||||
| 
 | ||||
|         sourceTask.stop(); | ||||
|         EasyMock.expectLastCall(); | ||||
|         expectOffsetFlush(true); | ||||
| 
 | ||||
|         PowerMock.replayAll(); | ||||
| 
 | ||||
|  | @ -450,6 +449,8 @@ public class WorkerSourceTaskTest extends ThreadedTest { | |||
|         IExpectationSetters<Void> futureGetExpect = EasyMock.expect( | ||||
|                 flushFuture.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))); | ||||
|         if (succeed) { | ||||
|             sourceTask.commit(); | ||||
|             EasyMock.expectLastCall(); | ||||
|             futureGetExpect.andReturn(null); | ||||
|         } else { | ||||
|             futureGetExpect.andThrow(new TimeoutException()); | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue