|
@@ -27,11 +27,15 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
|
|
|
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
|
|
|
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector.AggregationStatusTable;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import com.google.common.collect.Sets;
|
|
|
+import org.mockito.internal.stubbing.answers.AnswersWithDelay;
|
|
|
+import org.mockito.internal.stubbing.answers.Returns;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.HashSet;
|
|
@@ -46,6 +50,7 @@ import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.never;
|
|
|
import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
|
|
|
public class TestTimelineCollector {
|
|
|
|
|
@@ -165,17 +170,49 @@ public class TestTimelineCollector {
|
|
|
* putEntityAsync() calls.
|
|
|
*/
|
|
|
@Test
|
|
|
- public void testPutEntityAsync() throws IOException {
|
|
|
+ public void testPutEntityAsync() throws Exception {
|
|
|
TimelineWriter writer = mock(TimelineWriter.class);
|
|
|
TimelineCollector collector = new TimelineCollectorForTest(writer);
|
|
|
-
|
|
|
+ collector.init(new Configuration());
|
|
|
+ collector.start();
|
|
|
TimelineEntities entities = generateTestEntities(1, 1);
|
|
|
collector.putEntitiesAsync(
|
|
|
entities, UserGroupInformation.createRemoteUser("test-user"));
|
|
|
-
|
|
|
+ Thread.sleep(1000);
|
|
|
verify(writer, times(1)).write(any(TimelineCollectorContext.class),
|
|
|
any(TimelineEntities.class), any(UserGroupInformation.class));
|
|
|
verify(writer, never()).flush();
|
|
|
+ collector.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test TimelineCollector's discarding entities in case of async writes if
|
|
|
+ * write is taking too much time.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testAsyncEntityDiscard() throws Exception {
|
|
|
+ TimelineWriter writer = mock(TimelineWriter.class);
|
|
|
+
|
|
|
+ when(writer.write(any(), any(), any())).thenAnswer(
|
|
|
+ new AnswersWithDelay(500, new Returns(new TimelineWriteResponse())));
|
|
|
+ TimelineCollector collector = new TimelineCollectorForTest(writer);
|
|
|
+ Configuration config = new Configuration();
|
|
|
+ config
|
|
|
+ .setInt(YarnConfiguration.TIMELINE_SERVICE_WRITER_ASYNC_QUEUE_CAPACITY,
|
|
|
+ 3);
|
|
|
+ collector.init(config);
|
|
|
+ collector.start();
|
|
|
+ for (int i = 0; i < 10; ++i) {
|
|
|
+ TimelineEntities entities = generateTestEntities(i + 1, 1);
|
|
|
+ collector.putEntitiesAsync(entities,
|
|
|
+ UserGroupInformation.createRemoteUser("test-user"));
|
|
|
+ }
|
|
|
+ Thread.sleep(3000);
|
|
|
+ verify(writer, times(4))
|
|
|
+ .write(any(TimelineCollectorContext.class), any(TimelineEntities.class),
|
|
|
+ any(UserGroupInformation.class));
|
|
|
+ verify(writer, never()).flush();
|
|
|
+ collector.stop();
|
|
|
}
|
|
|
|
|
|
/**
|