diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/MemoryApplicationTimelineStore.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/MemoryApplicationTimelineStore.java new file mode 100644 index 0000000..41622d0 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/MemoryApplicationTimelineStore.java @@ -0,0 +1,263 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Set; +import java.util.SortedSet; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.service.AbstractService; + +/** + * In-memory implementation of {@link ApplicationTimelineStore}. + * This implementation is for test purpose only. If users improperly instantiate + * it, they may encounter reading and writing history data in different memory + * store. + * + */ +@Private +@Unstable +public class MemoryApplicationTimelineStore + extends AbstractService implements ApplicationTimelineStore { + + private Map entities = new HashMap(); + + public MemoryApplicationTimelineStore() { + super(MemoryApplicationTimelineStore.class.getName()); + } + + @Override + public List getEntities(String entityType, Long limit, + Long windowStart, Long windowEnd, NameValuePair primaryFilter, + Collection secondaryFilters, EnumSet fields) { + if (limit == null) { + limit = DEFAULT_LIMIT; + } + if (windowStart == null) { + windowStart = Long.MIN_VALUE; + } + if (windowEnd == null) { + windowEnd = Long.MAX_VALUE; + } + if (fields == null) { + fields = EnumSet.allOf(Field.class); + } + List entitiesSelected = new ArrayList(); + for (EntityInfo entityInfo : + new PriorityQueue(entities.values())) { + if (entitiesSelected.size() >= limit) { + break; + } + if (!entityInfo.getEntity().getType().equals(entityType)) { + continue; + } + if (entityInfo.getKeyTs() <= windowStart) { + continue; + } + if (entityInfo.getKeyTs() > windowEnd) { + continue; + } + if (primaryFilter != null && + !matchFilter(entityInfo.getPrimaryFilters(), primaryFilter)) { + continue; + } + if (secondaryFilters != null) { // OR logic + boolean flag = false; + for (NameValuePair secondaryFilter : secondaryFilters) { + if (secondaryFilter != null && + matchFilter(entityInfo.getOtherInfo(), secondaryFilter)) { + flag = true; + break; + } + } + if (!flag) { + continue; + } + } + entitiesSelected.add(entityInfo); + } + List entitiesToReturn = new ArrayList(); + for (EntityInfo entitySelected : entitiesSelected) { + entitiesToReturn.add(maskFields(entitySelected, fields)); + } + Collections.sort(entitiesToReturn); + return entitiesToReturn; + } + + @Override + public EntityInfo getEntityInfo(Entity entity, EnumSet fields) { + if (fields == null) { + fields = EnumSet.allOf(Field.class); + } + EntityInfo entityInfo = entities.get(entity); + if (entityInfo == null) { + return null; + } else { + return maskFields(entityInfo, fields); + } + } + + @Override + public Map> getEntityTimelines(String entityType, + SortedSet entityIds, Long limit, Long windowStart, Long windowEnd, + Set eventTypes) { + Map> allEvents = new HashMap>(); + if (entityIds == null) { + return allEvents; + } + if (limit == null) { + limit = DEFAULT_LIMIT; + } + if (windowStart == null) { + windowStart = Long.MIN_VALUE; + } + if (windowEnd == null) { + windowEnd = Long.MAX_VALUE; + } + for (String entityId : entityIds) { + Entity entity = new Entity(entityId, entityType); + EntityInfo entityInfo = entities.get(entity); + if (entityInfo == null) { + continue; + } + List events = new ArrayList(); + for (Event event : entityInfo.getEvents()) { + if (events.size() >= limit) { + break; + } + if (event.getTs() <= windowStart) { + continue; + } + if (event.getTs() > windowEnd) { + continue; + } + if (eventTypes != null && !eventTypes.contains(event.getType())) { + continue; + } + events.add(event); + } + allEvents.put(entity, events); + } + return allEvents; + } + + @Override + public TimelinePutResponse put(List data) { + TimelinePutResponse response = new TimelinePutResponse(); + for (TimelinePutRequest request : data) { + // construct entity + Entity entity = new Entity(request.getEntity(), request.getEntitytype()); + // store entity info in memory + EntityInfo existingEntityInfo = entities.get(entity); + if (existingEntityInfo == null) { + // construct entity info + EntityInfo entityInfo = new EntityInfo( + entity, + request.getKeyTs(), + request.getEvents() == null ? + new ArrayList() : new ArrayList(request.getEvents()), + new HashMap>(), + request.getPrimaryFilters() == null ? + new HashMap() : request.getPrimaryFilters(), + request.getOtherInfo() == null ? + new HashMap() : request.getOtherInfo()); + Collections.sort(entityInfo.getEvents()); + // check keyTs + if (entityInfo.getKeyTs() == null) { + if (entityInfo.getEvents().isEmpty()) { + response.addError(entity, TimelinePutResponse.Error.NO_KEYTS); + continue; + } else { + entityInfo.keyTs = entityInfo.getEvents().get(0).getTs(); + } + } + entities.put(entity, entityInfo); + } else { + if (request.getEvents() != null) { + existingEntityInfo.getEvents().addAll(request.getEvents()); + Collections.sort(existingEntityInfo.getEvents()); + } + if (request.getPrimaryFilters() != null) { + existingEntityInfo.getPrimaryFilters().putAll( + request.getPrimaryFilters()); + } + if (request.getOtherInfo() != null) { + existingEntityInfo.getOtherInfo().putAll(request.getOtherInfo()); + } + } + // relate it to other entities + if (request.getRelatedEntities() == null) { + continue; + } + for (Entity relatedEntity : request.getRelatedEntities()) { + EntityInfo relatedEntityInfo = entities.get(relatedEntity); + if (relatedEntityInfo == null) { + continue; + } + Map> relateEntities = + relatedEntityInfo.getRelatedEntities(); + List partRelateEntities = relateEntities.get(entity.getType()); + if (partRelateEntities == null) { + partRelateEntities = new ArrayList(); + relateEntities.put(entity.getType(), partRelateEntities); + } + if (!partRelateEntities.contains(entity.getId())) { + partRelateEntities.add(entity.getId()); + } + } + } + return response; + } + + private static EntityInfo maskFields( + EntityInfo entityInfo, EnumSet fields) { + return new EntityInfo( + entityInfo.getEntity(), + entityInfo.getKeyTs(), + fields.contains(Field.EVENTS) ? + entityInfo.getEvents() : fields.contains(Field.LASTEVENTONLY) ? + Arrays.asList(entityInfo.getEvents().get(0)) : null, + fields.contains(Field.RELATEDENTITIES) ? + entityInfo.getRelatedEntities() : null, + fields.contains(Field.PRIMARYFILTERS) ? + entityInfo.getPrimaryFilters() : null, + fields.contains(Field.OTHERINFO) ? + entityInfo.getOtherInfo() : null); + } + + private static boolean matchFilter(Map tags, NameValuePair filter) { + Object value = tags.get(filter.getName()); + if (value == null) { // doesn't have the filter + return false; + } else if (!value.equals(filter.getValue())) { // doesn't match the filter + return false; + } + return true; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/ApplicationTimelineStoreTestUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/ApplicationTimelineStoreTestUtils.java new file mode 100644 index 0000000..b94c6b4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/ApplicationTimelineStoreTestUtils.java @@ -0,0 +1,390 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; + +import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineReader.Field; + +public class ApplicationTimelineStoreTestUtils { + + private static final Map EMPTY_MAP = Collections.emptyMap(); + private static final Map> EMPTY_REL_ENTITIES = + new HashMap>(); + + public static void putEvents(ApplicationTimelineStore store) + throws IOException { + Map primaryFilters = new HashMap(); + primaryFilters.put("user", "username"); + primaryFilters.put("appname", 12345l); + Map secondaryFilters = new HashMap(); + secondaryFilters.put("startTime", 123456l); + secondaryFilters.put("status", "RUNNING"); + Map otherInfo1 = new HashMap(); + otherInfo1.put("info1", "val1"); + otherInfo1.putAll(secondaryFilters); + + Entity entity1 = new Entity("id_1", "type_1"); + Entity entity1b = new Entity("id_2", "type_1"); + Entity entity2 = new Entity("id_2", "type_2"); + + Event ev3 = new Event(789l, "launch_event"); + Event ev4 = new Event(-123l, "init_event"); + List events = new ArrayList(); + events.add(ev3); + events.add(ev4); + store.put(Collections.singletonList(TimelinePutRequest.newInstance(entity2, + null, events, null, null, null))); + + Event ev1 = new Event(123l, "start_event"); + store.put(Collections.singletonList(TimelinePutRequest.newInstance(entity1, + 123l, Collections.singletonList(ev1), Collections.singleton(entity2), + primaryFilters, otherInfo1))); + store.put(Collections.singletonList(TimelinePutRequest.newInstance( + entity1b, null, Collections.singletonList(ev1), + Collections.singleton(entity2), primaryFilters, otherInfo1))); + + Map eventInfo = new HashMap(); + eventInfo.put("event info 1", "val1"); + Event ev2 = new Event(456l, "end_event", eventInfo); + Map otherInfo2 = new HashMap(); + otherInfo2.put("info2", "val2"); + store.put(Collections.singletonList(TimelinePutRequest.newInstance(entity1, + null, Collections.singletonList(ev2), null, primaryFilters, + otherInfo2))); + store.put(Collections.singletonList(TimelinePutRequest.newInstance( + entity1b, 123l, Collections.singletonList(ev2), null, primaryFilters, + otherInfo2))); + } + + protected ApplicationTimelineStore store; + + public void test() throws IOException { + NameValuePair userFilter = new NameValuePair("user", + "username"); + Collection goodTestingFilters = + new ArrayList(); + goodTestingFilters.add(new NameValuePair("appname", 12345l)); + goodTestingFilters.add(new NameValuePair("status", "RUNNING")); + Collection badTestingFilters = + new ArrayList(); + badTestingFilters.add(new NameValuePair("appname", 12345l)); + badTestingFilters.add(new NameValuePair("status", "FINISHED")); + + Map primaryFilters = new HashMap(); + primaryFilters.put("user", "username"); + primaryFilters.put("appname", 12345l); + Map secondaryFilters = new HashMap(); + secondaryFilters.put("startTime", 123456l); + secondaryFilters.put("status", "RUNNING"); + Map allFilters = new HashMap(); + allFilters.putAll(secondaryFilters); + allFilters.putAll(primaryFilters); + Map otherInfo = new HashMap(); + otherInfo.put("info1", "val1"); + otherInfo.put("info2", "val2"); + otherInfo.putAll(secondaryFilters); + + Entity entity1 = new Entity("id_1", "type_1"); + Entity entity1b = new Entity("id_2", "type_1"); + Event ev1 = new Event(123l, "start_event"); + + Map eventInfo = new HashMap(); + eventInfo.put("event info 1", "val1"); + Event ev2 = new Event(456l, "end_event", eventInfo); + List events1 = new ArrayList(); + events1.add(ev2); + events1.add(ev1); + + Map> relEntityMap = + new HashMap>(); + List ids = new ArrayList(); + ids.add(entity1.getId()); + ids.add(entity1b.getId()); + relEntityMap.put(entity1.getType(), ids); + + Entity entity2 = new Entity("id_2", "type_2"); + Event ev3 = new Event(789l, "launch_event"); + Event ev4 = new Event(-123l, "init_event"); + List events2 = new ArrayList(); + events2.add(ev3); + events2.add(ev4); + + // test getting entity info + verifyEntityInfo(null, null, null, null, null, store.getEntityInfo( + new Entity("id_1", "type_2"), EnumSet.allOf(Field.class))); + + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, store.getEntityInfo(entity1, EnumSet.allOf(Field.class))); + + verifyEntityInfo(entity1b, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, store.getEntityInfo(entity1b, EnumSet.allOf(Field.class))); + + verifyEntityInfo(entity2, events2, relEntityMap, EMPTY_MAP, EMPTY_MAP, + store.getEntityInfo(entity2, EnumSet.allOf(Field.class))); + + // test getting single fields + verifyEntityInfo(entity1, events1, null, null, null, + store.getEntityInfo(entity1, EnumSet.of(Field.EVENTS))); + + verifyEntityInfo(entity1, Collections.singletonList(ev2), null, null, null, + store.getEntityInfo(entity1, EnumSet.of(Field.LASTEVENTONLY))); + + verifyEntityInfo(entity1, null, null, primaryFilters, null, + store.getEntityInfo(entity1, EnumSet.of(Field.PRIMARYFILTERS))); + + verifyEntityInfo(entity1, null, null, null, otherInfo, + store.getEntityInfo(entity1, EnumSet.of(Field.OTHERINFO))); + + verifyEntityInfo(entity2, null, relEntityMap, null, null, + store.getEntityInfo(entity2, EnumSet.of(Field.RELATEDENTITIES))); + + // test getting entities + assertEquals("nonzero entities size for nonexistent type", 0, + store.getEntities("type_0", null, null, null, null, null, + null).size()); + assertEquals("nonzero entities size for nonexistent type", 0, + store.getEntities("type_3", null, null, null, null, null, + null).size()); + assertEquals("nonzero entities size for nonexistent type", 0, + store.getEntities("type_0", null, null, null, userFilter, + null, null).size()); + assertEquals("nonzero entities size for nonexistent type", 0, + store.getEntities("type_3", null, null, null, userFilter, + null, null).size()); + + List entities = + store.getEntities("type_1", null, null, null, null, null, + EnumSet.allOf(Field.class)); + assertEquals(2, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + verifyEntityInfo(entity1b, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(1)); + + entities = store.getEntities("type_2", null, null, null, null, null, + EnumSet.allOf(Field.class)); + assertEquals(1, entities.size()); + verifyEntityInfo(entity2, events2, relEntityMap, EMPTY_MAP, EMPTY_MAP, + entities.get(0)); + + entities = store.getEntities("type_1", 1l, null, null, null, null, + EnumSet.allOf(Field.class)); + assertEquals(1, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + + entities = store.getEntities("type_1", 1l, 0l, null, null, null, + EnumSet.allOf(Field.class)); + assertEquals(1, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + + entities = store.getEntities("type_1", null, 234l, null, null, null, + EnumSet.allOf(Field.class)); + assertEquals(0, entities.size()); + + entities = store.getEntities("type_1", null, 234l, 345l, null, null, + EnumSet.allOf(Field.class)); + assertEquals(0, entities.size()); + + entities = store.getEntities("type_1", null, null, 345l, null, null, + EnumSet.allOf(Field.class)); + assertEquals(2, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + verifyEntityInfo(entity1b, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(1)); + + // test using primary filter + assertEquals("nonzero entities size for primary filter", 0, + store.getEntities("type_1", null, null, null, + new NameValuePair("none", "none"), null, + EnumSet.allOf(Field.class)).size()); + assertEquals("nonzero entities size for primary filter", 0, + store.getEntities("type_2", null, null, null, + new NameValuePair("none", "none"), null, + EnumSet.allOf(Field.class)).size()); + assertEquals("nonzero entities size for primary filter", 0, + store.getEntities("type_3", null, null, null, + new NameValuePair("none", "none"), null, + EnumSet.allOf(Field.class)).size()); + + entities = store.getEntities("type_1", null, null, null, userFilter, null, + EnumSet.allOf(Field.class)); + assertEquals(2, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + verifyEntityInfo(entity1b, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(1)); + + entities = store.getEntities("type_2", null, null, null, userFilter, null, + EnumSet.allOf(Field.class)); + assertEquals(0, entities.size()); + + entities = store.getEntities("type_1", 1l, null, null, userFilter, null, + EnumSet.allOf(Field.class)); + assertEquals(1, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + + entities = store.getEntities("type_1", 1l, 0l, null, userFilter, null, + EnumSet.allOf(Field.class)); + assertEquals(1, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + + entities = store.getEntities("type_1", null, 234l, null, userFilter, null, + EnumSet.allOf(Field.class)); + assertEquals(0, entities.size()); + + entities = store.getEntities("type_1", null, 234l, 345l, userFilter, null, + EnumSet.allOf(Field.class)); + assertEquals(0, entities.size()); + + entities = store.getEntities("type_1", null, null, 345l, userFilter, null, + EnumSet.allOf(Field.class)); + assertEquals(2, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + verifyEntityInfo(entity1b, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(1)); + + // test using secondary filter + entities = store.getEntities("type_1", null, null, null, null, + goodTestingFilters, EnumSet.allOf(Field.class)); + assertEquals(2, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + verifyEntityInfo(entity1b, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(1)); + + entities = store.getEntities("type_1", null, null, null, userFilter, + goodTestingFilters, EnumSet.allOf(Field.class)); + assertEquals(2, entities.size()); + verifyEntityInfo(entity1, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(0)); + verifyEntityInfo(entity1b, events1, EMPTY_REL_ENTITIES, primaryFilters, + otherInfo, entities.get(1)); + + entities = store.getEntities("type_1", null, null, null, null, + badTestingFilters, EnumSet.allOf(Field.class)); + assertEquals(0, entities.size()); + + entities = store.getEntities("type_1", null, null, null, userFilter, + badTestingFilters, EnumSet.allOf(Field.class)); + assertEquals(0, entities.size()); + + // test getting entity timelines + SortedSet sortedSet = new TreeSet(); + sortedSet.add(entity1.getId()); + Map> timelines = + store.getEntityTimelines(entity1.getType(), sortedSet, null, null, + null, null); + assertEquals(1, timelines.size()); + verifyEntityTimeline(timelines.get(entity1), ev2, ev1); + + sortedSet.add(entity1b.getId()); + timelines = store.getEntityTimelines(entity1.getType(), sortedSet, null, + null, null, null); + assertEquals(2, timelines.size()); + verifyEntityTimeline(timelines.get(entity1), ev2, ev1); + verifyEntityTimeline(timelines.get(entity1b), ev2, ev1); + + timelines = store.getEntityTimelines(entity1.getType(), sortedSet, 1l, + null, null, null); + assertEquals(2, timelines.size()); + verifyEntityTimeline(timelines.get(entity1), ev2); + verifyEntityTimeline(timelines.get(entity1b), ev2); + + timelines = store.getEntityTimelines(entity1.getType(), sortedSet, null, + 345l, null, null); + assertEquals(2, timelines.size()); + verifyEntityTimeline(timelines.get(entity1), ev2); + verifyEntityTimeline(timelines.get(entity1b), ev2); + + timelines = store.getEntityTimelines(entity1.getType(), sortedSet, null, + null, 345l, null); + assertEquals(2, timelines.size()); + verifyEntityTimeline(timelines.get(entity1), ev1); + verifyEntityTimeline(timelines.get(entity1b), ev1); + + timelines = store.getEntityTimelines(entity1.getType(), sortedSet, null, + null, null, Collections.singleton("end_event")); + assertEquals(2, timelines.size()); + verifyEntityTimeline(timelines.get(entity1), ev2); + verifyEntityTimeline(timelines.get(entity1b), ev2); + + sortedSet.add(entity2.getId()); + timelines = store.getEntityTimelines(entity2.getType(), sortedSet, null, + null, null, null); + assertEquals(1, timelines.size()); + verifyEntityTimeline(timelines.get(entity2), ev3, ev4); + } + + private void verifyEntityInfo(Entity entity, List events, + Map> relatedEntities, + Map primaryFilters, Map otherInfo, + EntityInfo retrievedEntityInfo) { + if (entity == null) { + assertNull(retrievedEntityInfo); + return; + } + assertEquals(entity, retrievedEntityInfo.getEntity()); + if (events == null) + assertNull(retrievedEntityInfo.getEvents()); + else + assertEquals(events, retrievedEntityInfo.getEvents()); + if (relatedEntities == null) + assertNull(retrievedEntityInfo.getRelatedEntities()); + else + assertEquals(relatedEntities, retrievedEntityInfo.getRelatedEntities()); + if (primaryFilters == null) + assertNull(retrievedEntityInfo.getPrimaryFilters()); + else + assertTrue(ATSUtil.compare(primaryFilters, + retrievedEntityInfo.getPrimaryFilters())); + if (otherInfo == null) + assertNull(retrievedEntityInfo.getOtherInfo()); + else + assertTrue(ATSUtil.compare(otherInfo, + retrievedEntityInfo.getOtherInfo())); + } + + private static void verifyEntityTimeline(List retrievedEvents, + Event... actualEvents) { + assertEquals(actualEvents.length, retrievedEvents.size()); + for (int i = 0; i < actualEvents.length; i++) { + assertEquals(actualEvents[i], retrievedEvents.get(i)); + } + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/TestMemoryApplicationTimelineStore.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/TestMemoryApplicationTimelineStore.java new file mode 100644 index 0000000..272b4e7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/TestMemoryApplicationTimelineStore.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline; + +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + + +public class TestMemoryApplicationTimelineStore + extends ApplicationTimelineStoreTestUtils { + + @Before + public void setup() throws Exception { + store = new MemoryApplicationTimelineStore(); + store.init(new YarnConfiguration()); + store.start(); + putEvents(store); + } + + @After + public void tearDown() throws Exception { + store.stop(); + } + + public void putEvents() throws Exception { + } + + public ApplicationTimelineStore getApplicationTimelineStore() { + return store; + } + + @Test + public void testApplicationTimelineStore() throws Exception { + test(); + } + +}