YARN-4446. Refactor reader API for better extensibility (Varun Saxena via sjlee)

This commit is contained in:
Sangjin Lee 2016-02-03 16:03:55 -08:00
parent 06f0b50a28
commit 9cb1287e9b
20 changed files with 1369 additions and 1090 deletions

View File

@ -0,0 +1,119 @@
/**
* 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.timelineservice.reader;
import java.util.EnumSet;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
/**
* Encapsulates information regarding which data to retrieve for each entity
* while querying.<br>
* Data to retrieve contains the following :<br>
* <ul>
* <li><b>confsToRetrieve</b> - Used for deciding which configs to return
* in response. This is represented as a {@link TimelineFilterList} object
* containing {@link TimelinePrefixFilter} objects. These can either be
* exact config keys' or prefixes which are then compared against config
* keys' to decide configs(inside entities) to return in response. If null
* or empty, all configurations will be fetched if fieldsToRetrieve
* contains {@link Field#CONFIGS} or {@link Field#ALL}. This should not be
* confused with configFilters which is used to decide which entities to
* return instead.</li>
* <li><b>metricsToRetrieve</b> - Used for deciding which metrics to return
* in response. This is represented as a {@link TimelineFilterList} object
* containing {@link TimelinePrefixFilter} objects. These can either be
* exact metric ids' or prefixes which are then compared against metric
* ids' to decide metrics(inside entities) to return in response. If null
* or empty, all metrics will be fetched if fieldsToRetrieve contains
* {@link Field#METRICS} or {@link Field#ALL}. This should not be confused
* with metricFilters which is used to decide which entities to return
* instead.</li>
* <li><b>fieldsToRetrieve</b> - Specifies which fields of the entity
* object to retrieve, see {@link Field}. If null, retrieves 3 fields,
* namely entity id, entity type and entity created time. All fields will
* be returned if {@link Field#ALL} is specified.</li>
* </ul>
*/
@Private
@Unstable
public class TimelineDataToRetrieve {
private TimelineFilterList confsToRetrieve;
private TimelineFilterList metricsToRetrieve;
private EnumSet<Field> fieldsToRetrieve;
public TimelineDataToRetrieve() {
this(null, null, null);
}
public TimelineDataToRetrieve(TimelineFilterList confs,
TimelineFilterList metrics, EnumSet<Field> fields) {
this.confsToRetrieve = confs;
this.metricsToRetrieve = metrics;
this.fieldsToRetrieve = fields;
if (this.fieldsToRetrieve == null) {
this.fieldsToRetrieve = EnumSet.noneOf(Field.class);
}
}
public TimelineFilterList getConfsToRetrieve() {
return confsToRetrieve;
}
public void setConfsToRetrieve(TimelineFilterList confs) {
this.confsToRetrieve = confs;
}
public TimelineFilterList getMetricsToRetrieve() {
return metricsToRetrieve;
}
public void setMetricsToRetrieve(TimelineFilterList metrics) {
this.metricsToRetrieve = metrics;
}
public EnumSet<Field> getFieldsToRetrieve() {
return fieldsToRetrieve;
}
public void setFieldsToRetrieve(EnumSet<Field> fields) {
this.fieldsToRetrieve = fields;
}
/**
* Adds configs and metrics fields to fieldsToRetrieve(if they are not
* present) if confsToRetrieve and metricsToRetrieve are specified.
*/
public void addFieldsBasedOnConfsAndMetricsToRetrieve() {
if (!fieldsToRetrieve.contains(Field.CONFIGS) && confsToRetrieve != null &&
!confsToRetrieve.getFilterList().isEmpty()) {
fieldsToRetrieve.add(Field.CONFIGS);
}
if (!fieldsToRetrieve.contains(Field.METRICS) &&
metricsToRetrieve != null &&
!metricsToRetrieve.getFilterList().isEmpty()) {
fieldsToRetrieve.add(Field.METRICS);
}
}
}

View File

@ -0,0 +1,187 @@
/**
* 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.timelineservice.reader;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* Encapsulates information regarding the filters to apply while querying. These
* filters restrict the number of entities to return.<br>
* Filters contain the following :<br>
* <ul>
* <li><b>limit</b> - A limit on the number of entities to return. If null
* or {@literal <=0}, defaults to {@link #DEFAULT_LIMIT}.</li>
* <li><b>createdTimeBegin</b> - Matched entities should not be created
* before this timestamp. If null or {@literal <=0}, defaults to 0.</li>
* <li><b>createdTimeEnd</b> - Matched entities should not be created after
* this timestamp. If null or {@literal <=0}, defaults to
* {@link Long#MAX_VALUE}.</li>
* <li><b>relatesTo</b> - Matched entities should relate to given entities.
* If null or empty, the relations are not matched.</li>
* <li><b>isRelatedTo</b> - Matched entities should be related to given
* entities. If null or empty, the relations are not matched.</li>
* <li><b>infoFilters</b> - Matched entities should have exact matches to
* the given info represented as key-value pairs. If null or empty, the
* filter is not applied.</li>
* <li><b>configFilters</b> - Matched entities should have exact matches to
* the given configs represented as key-value pairs. If null or empty, the
* filter is not applied.</li>
* <li><b>metricFilters</b> - Matched entities should contain the given
* metrics. If null or empty, the filter is not applied.</li>
* <li><b>eventFilters</b> - Matched entities should contain the given
* events. If null or empty, the filter is not applied.</li>
* </ul>
*/
@Private
@Unstable
public class TimelineEntityFilters {
private Long limit;
private Long createdTimeBegin;
private Long createdTimeEnd;
private Map<String, Set<String>> relatesTo;
private Map<String, Set<String>> isRelatedTo;
private Map<String, Object> infoFilters;
private Map<String, String> configFilters;
private Set<String> metricFilters;
private Set<String> eventFilters;
private static final Long DEFAULT_BEGIN_TIME = 0L;
private static final Long DEFAULT_END_TIME = Long.MAX_VALUE;
/**
* Default limit of number of entities to return for getEntities API.
*/
public static final long DEFAULT_LIMIT = 100;
public TimelineEntityFilters() {
this(null, null, null, null, null, null, null, null, null);
}
public TimelineEntityFilters(
Long entityLimit, Long timeBegin, Long timeEnd,
Map<String, Set<String>> entityRelatesTo,
Map<String, Set<String>> entityIsRelatedTo,
Map<String, Object> entityInfoFilters,
Map<String, String> entityConfigFilters,
Set<String> entityMetricFilters,
Set<String> entityEventFilters) {
this.limit = entityLimit;
if (this.limit == null || this.limit < 0) {
this.limit = DEFAULT_LIMIT;
}
this.createdTimeBegin = timeBegin;
if (this.createdTimeBegin == null || this.createdTimeBegin < 0) {
this.createdTimeBegin = DEFAULT_BEGIN_TIME;
}
this.createdTimeEnd = timeEnd;
if (this.createdTimeEnd == null || this.createdTimeEnd < 0) {
this.createdTimeEnd = DEFAULT_END_TIME;
}
this.relatesTo = entityRelatesTo;
this.isRelatedTo = entityIsRelatedTo;
this.infoFilters = entityInfoFilters;
this.configFilters = entityConfigFilters;
this.metricFilters = entityMetricFilters;
this.eventFilters = entityEventFilters;
}
public Long getLimit() {
return limit;
}
public void setLimit(Long entityLimit) {
this.limit = entityLimit;
if (this.limit == null || this.limit < 0) {
this.limit = DEFAULT_LIMIT;
}
}
public Long getCreatedTimeBegin() {
return createdTimeBegin;
}
public void setCreatedTimeBegin(Long timeBegin) {
this.createdTimeBegin = timeBegin;
if (this.createdTimeBegin == null || this.createdTimeBegin < 0) {
this.createdTimeBegin = DEFAULT_BEGIN_TIME;
}
}
public Long getCreatedTimeEnd() {
return createdTimeEnd;
}
public void setCreatedTimeEnd(Long timeEnd) {
this.createdTimeEnd = timeEnd;
if (this.createdTimeEnd == null || this.createdTimeEnd < 0) {
this.createdTimeEnd = DEFAULT_END_TIME;
}
}
public Map<String, Set<String>> getRelatesTo() {
return relatesTo;
}
public void setRelatesTo(Map<String, Set<String>> relations) {
this.relatesTo = relations;
}
public Map<String, Set<String>> getIsRelatedTo() {
return isRelatedTo;
}
public void setIsRelatedTo(Map<String, Set<String>> relations) {
this.isRelatedTo = relations;
}
public Map<String, Object> getInfoFilters() {
return infoFilters;
}
public void setInfoFilters(Map<String, Object> filters) {
this.infoFilters = filters;
}
public Map<String, String> getConfigFilters() {
return configFilters;
}
public void setConfigFilters(Map<String, String> filters) {
this.configFilters = filters;
}
public Set<String> getMetricFilters() {
return metricFilters;
}
public void setMetricFilters(Set<String> filters) {
this.metricFilters = filters;
}
public Set<String> getEventFilters() {
return eventFilters;
}
public void setEventFilters(Set<String> filters) {
this.eventFilters = filters;
}
}

View File

@ -18,11 +18,15 @@
package org.apache.hadoop.yarn.server.timelineservice.reader; package org.apache.hadoop.yarn.server.timelineservice.reader;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.server.timelineservice.TimelineContext; import org.apache.hadoop.yarn.server.timelineservice.TimelineContext;
/** /**
* Encapsulates fields necessary to make a query in timeline reader. * Encapsulates fields necessary to make a query in timeline reader.
*/ */
@Private
@Unstable
public class TimelineReaderContext extends TimelineContext { public class TimelineReaderContext extends TimelineContext {
private String entityType; private String entityType;
@ -34,6 +38,12 @@ public TimelineReaderContext(String clusterId, String userId, String flowName,
this.entityId = entityId; this.entityId = entityId;
} }
public TimelineReaderContext(TimelineReaderContext other) {
this(other.getClusterId(), other.getUserId(), other.getFlowName(),
other.getFlowRunId(), other.getAppId(), other.getEntityType(),
other.getEntityId());
}
@Override @Override
public int hashCode() { public int hashCode() {
final int prime = 31; final int prime = 31;

View File

@ -19,8 +19,6 @@
package org.apache.hadoop.yarn.server.timelineservice.reader; package org.apache.hadoop.yarn.server.timelineservice.reader;
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet;
import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -33,7 +31,6 @@
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -122,23 +119,14 @@ private static void fillUID(TimelineEntityType entityType,
* *
* @see TimelineReader#getEntities * @see TimelineReader#getEntities
*/ */
public Set<TimelineEntity> getEntities(String userId, String clusterId, public Set<TimelineEntity> getEntities(TimelineReaderContext context,
String flowName, Long flowRunId, String appId, String entityType, TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve)
Long limit, Long createdTimeBegin, Long createdTimeEnd, throws IOException {
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo, context.setClusterId(getClusterID(context.getClusterId(), getConfig()));
Map<String, Object> infoFilters, Map<String, String> configFilters, Set<TimelineEntity> entities = reader.getEntities(
Set<String> metricFilters, Set<String> eventFilters, new TimelineReaderContext(context), filters, dataToRetrieve);
EnumSet<Field> fieldsToRetrieve) throws IOException {
String cluster = getClusterID(clusterId, getConfig());
Set<TimelineEntity> entities =
reader.getEntities(userId, cluster, flowName, flowRunId, appId,
entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo,
isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters,
null, null, fieldsToRetrieve);
if (entities != null) { if (entities != null) {
TimelineEntityType type = getTimelineEntityType(entityType); TimelineEntityType type = getTimelineEntityType(context.getEntityType());
TimelineReaderContext context = new TimelineReaderContext(cluster, userId,
flowName, flowRunId, appId, entityType, null);
for (TimelineEntity entity : entities) { for (TimelineEntity entity : entities) {
fillUID(type, entity, context); fillUID(type, entity, context);
} }
@ -152,18 +140,14 @@ public Set<TimelineEntity> getEntities(String userId, String clusterId,
* *
* @see TimelineReader#getEntity * @see TimelineReader#getEntity
*/ */
public TimelineEntity getEntity(String userId, String clusterId, public TimelineEntity getEntity(TimelineReaderContext context,
String flowName, Long flowRunId, String appId, String entityType, TimelineDataToRetrieve dataToRetrieve) throws IOException {
String entityId, EnumSet<Field> fields) throws IOException { context.setClusterId(
String cluster = getClusterID(clusterId, getConfig()); getClusterID(context.getClusterId(), getConfig()));
TimelineEntity entity = TimelineEntity entity = reader.getEntity(
reader.getEntity(userId, cluster, flowName, flowRunId, appId, new TimelineReaderContext(context), dataToRetrieve);
entityType, entityId, null, null, fields);
if (entity != null) { if (entity != null) {
TimelineEntityType type = getTimelineEntityType(entityType); TimelineEntityType type = getTimelineEntityType(context.getEntityType());
TimelineReaderContext context = new TimelineReaderContext(cluster, userId,
flowName, flowRunId, appId, entityType, null);
fillUID(type, entity, context); fillUID(type, entity, context);
} }
return entity; return entity;

View File

@ -90,7 +90,7 @@ static List<String> split(final String str, final char delimiterChar,
// String needs to be split here. Copy remaining chars and add the // String needs to be split here. Copy remaining chars and add the
// string to list. // string to list.
builder.append(str.substring(startOffset, offset)); builder.append(str.substring(startOffset, offset));
list.add(builder.toString()); list.add(builder.toString().trim());
// Reset the start offset as a delimiter has been encountered. // Reset the start offset as a delimiter has been encountered.
startOffset = ++offset; startOffset = ++offset;
builder = new StringBuilder(len - offset); builder = new StringBuilder(len - offset);
@ -103,7 +103,7 @@ static List<String> split(final String str, final char delimiterChar,
builder.append(str.substring(startOffset)); builder.append(str.substring(startOffset));
} }
// Add the last part of delimited string to list. // Add the last part of delimited string to list.
list.add(builder.toString()); list.add(builder.toString().trim());
return list; return list;
} }

View File

@ -46,8 +46,6 @@
import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout; import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; 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.TimelineEntityType;
@ -70,8 +68,6 @@ public class TimelineReaderWebServices {
@Context private ServletContext ctxt; @Context private ServletContext ctxt;
private static final String COMMA_DELIMITER = ",";
private static final String COLON_DELIMITER = ":";
private static final String QUERY_STRING_SEP = "?"; private static final String QUERY_STRING_SEP = "?";
private static final String RANGE_DELIMITER = "-"; private static final String RANGE_DELIMITER = "-";
private static final String DATE_PATTERN = "yyyyMMdd"; private static final String DATE_PATTERN = "yyyyMMdd";
@ -243,15 +239,15 @@ public TimelineAbout about(
* events. This is represented as eventfilters=eventid1, eventid2... * events. This is represented as eventfilters=eventid1, eventid2...
* @param fields Specifies which fields of the entity object to retrieve, see * @param fields Specifies which fields of the entity object to retrieve, see
* {@link Field}. All fields will be retrieved if fields=ALL. If not * {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type, id and created time is returned * specified, 3 fields i.e. entity type, id and created time is returned
* (Optional query param). * (Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing * @return If successful, a HTTP 200(OK) response having a JSON representing
* a set of {@link TimelineEntity} instances of the given entity type is * a set of <cite>TimelineEntity</cite> instances of the given entity type
* returned. * is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request or UID is incorrect, * If any problem occurs in parsing request or UID is incorrect,
* HTTP 400(Bad Request) is returned. * HTTP 400(Bad Request) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -290,30 +286,14 @@ public Set<TimelineEntity> getEntities(
if (context == null) { if (context == null) {
throw new BadRequestException("Incorrect UID " + uId); throw new BadRequestException("Incorrect UID " + uId);
} }
entities = timelineReaderManager.getEntities( context.setEntityType(
TimelineReaderWebServicesUtils.parseStr(context.getUserId()), TimelineReaderWebServicesUtils.parseStr(entityType));
TimelineReaderWebServicesUtils.parseStr(context.getClusterId()), entities = timelineReaderManager.getEntities(context,
TimelineReaderWebServicesUtils.parseStr(context.getFlowName()), TimelineReaderWebServicesUtils.createTimelineEntityFilters(
context.getFlowRunId(), limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
TimelineReaderWebServicesUtils.parseStr(context.getAppId()), infofilters, conffilters, metricfilters, eventfilters),
TimelineReaderWebServicesUtils.parseStr(entityType), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
TimelineReaderWebServicesUtils.parseLongStr(limit), null, null, fields));
TimelineReaderWebServicesUtils.parseLongStr(createdTimeStart),
TimelineReaderWebServicesUtils.parseLongStr(createdTimeEnd),
TimelineReaderWebServicesUtils.parseKeyStrValuesStr(
relatesTo, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValuesStr(
isRelatedTo, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValueObj(
infofilters, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValueStr(
conffilters, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseValuesStr(
metricfilters, COMMA_DELIMITER),
TimelineReaderWebServicesUtils.parseValuesStr(
eventfilters, COMMA_DELIMITER),
TimelineReaderWebServicesUtils.parseFieldsStr(
fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, handleException(e, url, startTime,
"createdTime start/end or limit or flowrunid"); "createdTime start/end or limit or flowrunid");
@ -374,17 +354,17 @@ public Set<TimelineEntity> getEntities(
* events. This is represented as eventfilters=eventid1, eventid2... * events. This is represented as eventfilters=eventid1, eventid2...
* @param fields Specifies which fields of the entity object to retrieve, see * @param fields Specifies which fields of the entity object to retrieve, see
* {@link Field}. All fields will be retrieved if fields=ALL. If not * {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type, id, created time is returned * specified, 3 fields i.e. entity type, id, created time is returned
* (Optional query param). * (Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing * @return If successful, a HTTP 200(OK) response having a JSON representing
* a set of {@link TimelineEntity} instances of the given entity type is * a set of <cite>TimelineEntity</cite> instances of the given entity type
* returned. * is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* If flow context information cannot be retrieved, HTTP 404(Not Found) * If flow context information cannot be retrieved, HTTP 404(Not Found)
* is returned. * is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -463,17 +443,17 @@ public Set<TimelineEntity> getEntities(
* events. This is represented as eventfilters=eventid1, eventid2... * events. This is represented as eventfilters=eventid1, eventid2...
* @param fields Specifies which fields of the entity object to retrieve, see * @param fields Specifies which fields of the entity object to retrieve, see
* {@link Field}. All fields will be retrieved if fields=ALL. If not * {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type, id, created time is returned * specified, 3 fields i.e. entity type, id, created time is returned
* (Optional query param). * (Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing * @return If successful, a HTTP 200(OK) response having a JSON representing
* a set of {@link TimelineEntity} instances of the given entity type is * a set of <cite>TimelineEntity</cite> instances of the given entity type
* returned. * is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* If flow context information cannot be retrieved, HTTP 404(Not Found) * If flow context information cannot be retrieved, HTTP 404(Not Found)
* is returned. * is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -512,29 +492,14 @@ public Set<TimelineEntity> getEntities(
Set<TimelineEntity> entities = null; Set<TimelineEntity> entities = null;
try { try {
entities = timelineReaderManager.getEntities( entities = timelineReaderManager.getEntities(
TimelineReaderWebServicesUtils.parseStr(userId), TimelineReaderWebServicesUtils.createTimelineReaderContext(
TimelineReaderWebServicesUtils.parseStr(clusterId), clusterId, userId, flowName, flowRunId, appId, entityType, null),
TimelineReaderWebServicesUtils.parseStr(flowName),
TimelineReaderWebServicesUtils.parseLongStr(flowRunId), TimelineReaderWebServicesUtils.createTimelineEntityFilters(
TimelineReaderWebServicesUtils.parseStr(appId), limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
TimelineReaderWebServicesUtils.parseStr(entityType), infofilters, conffilters, metricfilters, eventfilters),
TimelineReaderWebServicesUtils.parseLongStr(limit), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
TimelineReaderWebServicesUtils.parseLongStr(createdTimeStart), null, null, fields));
TimelineReaderWebServicesUtils.parseLongStr(createdTimeEnd),
TimelineReaderWebServicesUtils.parseKeyStrValuesStr(
relatesTo, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValuesStr(
isRelatedTo, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValueObj(
infofilters, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValueStr(
conffilters, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseValuesStr(
metricfilters, COMMA_DELIMITER),
TimelineReaderWebServicesUtils.parseValuesStr(
eventfilters, COMMA_DELIMITER),
TimelineReaderWebServicesUtils.parseFieldsStr(
fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, handleException(e, url, startTime,
"createdTime start/end or limit or flowrunid"); "createdTime start/end or limit or flowrunid");
@ -559,16 +524,16 @@ public Set<TimelineEntity> getEntities(
* UID and then used to query backend(Mandatory path param). * UID and then used to query backend(Mandatory path param).
* @param fields Specifies which fields of the entity object to retrieve, see * @param fields Specifies which fields of the entity object to retrieve, see
* {@link Field}. All fields will be retrieved if fields=ALL. If not * {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type, id, created time is returned * specified, 3 fields i.e. entity type, id, created time is returned
* (Optional query param). * (Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link TimelineEntity} instance is returned. * <cite>TimelineEntity</cite> instance is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request or UID is incorrect, * If any problem occurs in parsing request or UID is incorrect,
* HTTP 400(Bad Request) is returned. * HTTP 400(Bad Request) is returned.<br>
* If entity for the given entity id cannot be found, HTTP 404(Not Found) * If entity for the given entity id cannot be found, HTTP 404(Not Found)
* is returned. * is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -597,11 +562,9 @@ public TimelineEntity getEntity(
if (context == null) { if (context == null) {
throw new BadRequestException("Incorrect UID " + uId); throw new BadRequestException("Incorrect UID " + uId);
} }
entity = timelineReaderManager.getEntity(context.getUserId(), entity = timelineReaderManager.getEntity(context,
context.getClusterId(), context.getFlowName(), context.getFlowRunId(), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
context.getAppId(), context.getEntityType(), context.getEntityId(), null, null, fields));
TimelineReaderWebServicesUtils.parseFieldsStr(
fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "flowrunid"); handleException(e, url, startTime, "flowrunid");
} }
@ -638,16 +601,16 @@ public TimelineEntity getEntity(
* param). * param).
* @param fields Specifies which fields of the entity object to retrieve, see * @param fields Specifies which fields of the entity object to retrieve, see
* {@link Field}. All fields will be retrieved if fields=ALL. If not * {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type, id, created time is returned * specified, 3 fields i.e. entity type, id, created time is returned
* (Optional query param). * (Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link TimelineEntity} instance is returned. * <cite>TimelineEntity</cite> instance is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* If flow context information cannot be retrieved or entity for the given * If flow context information cannot be retrieved or entity for the given
* entity id cannot be found, HTTP 404(Not Found) is returned. * entity id cannot be found, HTTP 404(Not Found) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -690,16 +653,16 @@ public TimelineEntity getEntity(
* param). * param).
* @param fields Specifies which fields of the entity object to retrieve, see * @param fields Specifies which fields of the entity object to retrieve, see
* {@link Field}. All fields will be retrieved if fields=ALL. If not * {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type, id and created time is returned * specified, 3 fields i.e. entity type, id and created time is returned
* (Optional query param). * (Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link TimelineEntity} instance is returned. * <cite>TimelineEntity</cite> instance is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* If flow context information cannot be retrieved or entity for the given * If flow context information cannot be retrieved or entity for the given
* entity id cannot be found, HTTP 404(Not Found) is returned. * entity id cannot be found, HTTP 404(Not Found) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -728,17 +691,12 @@ public TimelineEntity getEntity(
init(res); init(res);
TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
TimelineEntity entity = null; TimelineEntity entity = null;
String type = TimelineReaderWebServicesUtils.parseStr(entityType);
String id = TimelineReaderWebServicesUtils.parseStr(entityId);
try { try {
entity = timelineReaderManager.getEntity( entity = timelineReaderManager.getEntity(
TimelineReaderWebServicesUtils.parseStr(userId), TimelineReaderWebServicesUtils.createTimelineReaderContext(
TimelineReaderWebServicesUtils.parseStr(clusterId), clusterId, userId, flowName, flowRunId, appId, entityType, entityId),
TimelineReaderWebServicesUtils.parseStr(flowName), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
TimelineReaderWebServicesUtils.parseLongStr(flowRunId), null, null, fields));
TimelineReaderWebServicesUtils.parseStr(appId), type, id,
TimelineReaderWebServicesUtils.parseFieldsStr(
fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "flowrunid"); handleException(e, url, startTime, "flowrunid");
} }
@ -746,8 +704,8 @@ public TimelineEntity getEntity(
if (entity == null) { if (entity == null) {
LOG.info("Processed URL " + url + " but entity not found" + " (Took " + LOG.info("Processed URL " + url + " but entity not found" + " (Took " +
(endTime - startTime) + " ms.)"); (endTime - startTime) + " ms.)");
throw new NotFoundException("Timeline entity {id: " + id + ", type: " + throw new NotFoundException("Timeline entity {id: " + entityId +
type + " } is not found"); ", type: " + entityType + " } is not found");
} }
LOG.info("Processed URL " + url + LOG.info("Processed URL " + url +
" (Took " + (endTime - startTime) + " ms.)"); " (Took " + (endTime - startTime) + " ms.)");
@ -765,13 +723,13 @@ public TimelineEntity getEntity(
* (Mandatory path param). * (Mandatory path param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link FlowRunEntity} instance is returned. By default, all metrics for * <cite>FlowRunEntity</cite> instance is returned. By default, all
* the flow run will be returned. * metrics for the flow run will be returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request or UID is incorrect, * If any problem occurs in parsing request or UID is incorrect,
* HTTP 400(Bad Request) is returned. * HTTP 400(Bad Request) is returned.<br>
* If flow run for the given flow run id cannot be found, HTTP 404 * If flow run for the given flow run id cannot be found, HTTP 404
* (Not Found) is returned. * (Not Found) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -799,9 +757,9 @@ public TimelineEntity getFlowRun(
if (context == null) { if (context == null) {
throw new BadRequestException("Incorrect UID " + uId); throw new BadRequestException("Incorrect UID " + uId);
} }
entity = timelineReaderManager.getEntity(context.getUserId(), context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString());
context.getClusterId(), context.getFlowName(), context.getFlowRunId(), entity = timelineReaderManager.getEntity(context,
null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null); new TimelineDataToRetrieve());
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "flowrunid"); handleException(e, url, startTime, "flowrunid");
} }
@ -829,13 +787,13 @@ public TimelineEntity getFlowRun(
* @param flowRunId Id of the flow run to be queried(Mandatory path param). * @param flowRunId Id of the flow run to be queried(Mandatory path param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link FlowRunEntity} instance is returned. By default, all metrics for * <cite>FlowRunEntity</cite> instance is returned. By default, all
* the flow run will be returned. * metrics for the flow run will be returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* If flow run for the given flow run id cannot be found, HTTP 404 * If flow run for the given flow run id cannot be found, HTTP 404
* (Not Found) is returned. * (Not Found) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -865,13 +823,13 @@ public TimelineEntity getFlowRun(
* @param flowRunId Id of the flow run to be queried(Mandatory path param). * @param flowRunId Id of the flow run to be queried(Mandatory path param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link FlowRunEntity} instance is returned. By default, all metrics for * <cite>FlowRunEntity</cite> instance is returned. By default, all
* the flow run will be returned. * metrics for the flow run will be returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* If flow run for the given flow run id cannot be found, HTTP 404 * If flow run for the given flow run id cannot be found, HTTP 404
* (Not Found) is returned. * (Not Found) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -899,11 +857,10 @@ public TimelineEntity getFlowRun(
TimelineEntity entity = null; TimelineEntity entity = null;
try { try {
entity = timelineReaderManager.getEntity( entity = timelineReaderManager.getEntity(
TimelineReaderWebServicesUtils.parseStr(userId), TimelineReaderWebServicesUtils.createTimelineReaderContext(
TimelineReaderWebServicesUtils.parseStr(clusterId), clusterId, userId, flowName, flowRunId, null,
TimelineReaderWebServicesUtils.parseStr(flowName), TimelineEntityType.YARN_FLOW_RUN.toString(), null),
TimelineReaderWebServicesUtils.parseLongStr(flowRunId), new TimelineDataToRetrieve());
null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null);
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "flowrunid"); handleException(e, url, startTime, "flowrunid");
} }
@ -930,8 +887,6 @@ public TimelineEntity getFlowRun(
* @param uId a delimited string containing clusterid, userid, and flow name * @param uId a delimited string containing clusterid, userid, and flow name
* which are extracted from UID and then used to query backend(Mandatory * which are extracted from UID and then used to query backend(Mandatory
* path param). * path param).
* @param flowName Flow name to which the flow runs to be queried belongs to(
* Mandatory path param).
* @param limit Number of flow runs to return(Optional query param). * @param limit Number of flow runs to return(Optional query param).
* @param createdTimeStart If specified, matched flow runs should not be * @param createdTimeStart If specified, matched flow runs should not be
* created before this timestamp(Optional query param). * created before this timestamp(Optional query param).
@ -943,10 +898,11 @@ public TimelineEntity getFlowRun(
* other than metrics are returned(Optional query param). * other than metrics are returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* set of {@link FlowRunEntity} instances for the given flow are returned. * set of <cite>FlowRunEntity</cite> instances for the given flow are
* On failures, * returned.<br>
* On failures,<br>
* If any problem occurs in parsing request or UID is incorrect, * If any problem occurs in parsing request or UID is incorrect,
* HTTP 400(Bad Request) is returned. * HTTP 400(Bad Request) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -978,14 +934,13 @@ public Set<TimelineEntity> getFlowRuns(
if (context == null) { if (context == null) {
throw new BadRequestException("Incorrect UID " + uId); throw new BadRequestException("Incorrect UID " + uId);
} }
entities = timelineReaderManager.getEntities(context.getUserId(), context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString());
context.getClusterId(), context.getFlowName(), null, null, entities = timelineReaderManager.getEntities(context,
TimelineEntityType.YARN_FLOW_RUN.toString(), TimelineReaderWebServicesUtils.createTimelineEntityFilters(
TimelineReaderWebServicesUtils.parseLongStr(limit), limit, createdTimeStart, createdTimeEnd, null, null, null,
TimelineReaderWebServicesUtils.parseLongStr(createdTimeStart), null, null, null),
TimelineReaderWebServicesUtils.parseLongStr(createdTimeEnd), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
null, null, null, null, null, null, TimelineReaderWebServicesUtils. null, null, fields));
parseFieldsStr(fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "createdTime start/end or limit"); handleException(e, url, startTime, "createdTime start/end or limit");
} }
@ -1019,10 +974,11 @@ public Set<TimelineEntity> getFlowRuns(
* other than metrics are returned(Optional query param). * other than metrics are returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* set of {@link FlowRunEntity} instances for the given flow are returned. * set of <cite>FlowRunEntity</cite> instances for the given flow are
* On failures, * returned.<br>
* On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1064,10 +1020,11 @@ public Set<TimelineEntity> getFlowRuns(
* other than metrics are returned(Optional query param). * other than metrics are returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* set of {@link FlowRunEntity} instances for the given flow are returned. * set of <cite>FlowRunEntity</cite> instances for the given flow are
* On failures, * returned.<br>
* On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1097,15 +1054,14 @@ public Set<TimelineEntity> getFlowRuns(
Set<TimelineEntity> entities = null; Set<TimelineEntity> entities = null;
try { try {
entities = timelineReaderManager.getEntities( entities = timelineReaderManager.getEntities(
TimelineReaderWebServicesUtils.parseStr(userId), TimelineReaderWebServicesUtils.createTimelineReaderContext(
TimelineReaderWebServicesUtils.parseStr(clusterId), clusterId, userId, flowName, null, null,
TimelineReaderWebServicesUtils.parseStr(flowName), null, null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
TimelineEntityType.YARN_FLOW_RUN.toString(), TimelineReaderWebServicesUtils.createTimelineEntityFilters(
TimelineReaderWebServicesUtils.parseLongStr(limit), limit, createdTimeStart, createdTimeEnd, null, null, null,
TimelineReaderWebServicesUtils.parseLongStr(createdTimeStart), null, null, null),
TimelineReaderWebServicesUtils.parseLongStr(createdTimeEnd), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
null, null, null, null, null, null, TimelineReaderWebServicesUtils. null, null, fields));
parseFieldsStr(fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "createdTime start/end or limit"); handleException(e, url, startTime, "createdTime start/end or limit");
} }
@ -1142,12 +1098,12 @@ public Set<TimelineEntity> getFlowRuns(
* "daterange=-20150711" returns flows active on and before 20150711. * "daterange=-20150711" returns flows active on and before 20150711.
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* set of {@link FlowActivityEntity} instances are returned. * set of <cite>FlowActivityEntity</cite> instances are returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.<br>
*/ */
@GET @GET
@Path("/flows/") @Path("/flows/")
@ -1185,10 +1141,10 @@ public Set<TimelineEntity> getFlows(
* "daterange=-20150711" returns flows active on and before 20150711. * "daterange=-20150711" returns flows active on and before 20150711.
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* set of {@link FlowActivityEntity} instances are returned. * set of <cite>FlowActivityEntity</cite> instances are returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1214,11 +1170,17 @@ public Set<TimelineEntity> getFlows(
Set<TimelineEntity> entities = null; Set<TimelineEntity> entities = null;
try { try {
DateRange range = parseDateRange(dateRange); DateRange range = parseDateRange(dateRange);
TimelineEntityFilters entityFilters =
TimelineReaderWebServicesUtils.createTimelineEntityFilters(
limit, null, null, null, null, null, null, null, null);
entityFilters.setCreatedTimeBegin(range.dateStart);
entityFilters.setCreatedTimeEnd(range.dateEnd);
entities = timelineReaderManager.getEntities( entities = timelineReaderManager.getEntities(
null, TimelineReaderWebServicesUtils.parseStr(clusterId), null, null, TimelineReaderWebServicesUtils.createTimelineReaderContext(
null, TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), clusterId, null, null, null, null,
TimelineReaderWebServicesUtils.parseLongStr(limit), range.dateStart, TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
range.dateEnd, null, null, null, null, null, null, null); entityFilters, TimelineReaderWebServicesUtils.
createTimelineDataToRetrieve(null, null, null));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "limit"); handleException(e, url, startTime, "limit");
} }
@ -1242,16 +1204,16 @@ public Set<TimelineEntity> getFlows(
* backend(Mandatory path param). * backend(Mandatory path param).
* @param fields Specifies which fields of the app entity object to retrieve, * @param fields Specifies which fields of the app entity object to retrieve,
* see {@link Field}. All fields will be retrieved if fields=ALL. If not * see {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
* app id and app created time is returned(Optional query param). * app id and app created time is returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link TimelineEntity} instance is returned. * <cite>TimelineEntity</cite> instance is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request or UID is incorrect, * If any problem occurs in parsing request or UID is incorrect,
* HTTP 400(Bad Request) is returned. * HTTP 400(Bad Request) is returned.<br>
* If app for the given app id cannot be found, HTTP 404(Not Found) is * If app for the given app id cannot be found, HTTP 404(Not Found) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1280,11 +1242,10 @@ public TimelineEntity getApp(
if (context == null) { if (context == null) {
throw new BadRequestException("Incorrect UID " + uId); throw new BadRequestException("Incorrect UID " + uId);
} }
entity = timelineReaderManager.getEntity(context.getUserId(), context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString());
context.getClusterId(), context.getFlowName(), context.getFlowRunId(), entity = timelineReaderManager.getEntity(context,
context.getAppId(), TimelineEntityType.YARN_APPLICATION.toString(), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
null, TimelineReaderWebServicesUtils.parseFieldsStr( null, null, fields));
fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "flowrunid"); handleException(e, url, startTime, "flowrunid");
} }
@ -1316,16 +1277,16 @@ public TimelineEntity getApp(
* @param userId User id which should match for the app(Optional query param). * @param userId User id which should match for the app(Optional query param).
* @param fields Specifies which fields of the app entity object to retrieve, * @param fields Specifies which fields of the app entity object to retrieve,
* see {@link Field}. All fields will be retrieved if fields=ALL. If not * see {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
* app id and app created time is returned(Optional query param). * app id and app created time is returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link TimelineEntity} instance is returned. * <cite>TimelineEntity</cite> instance is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* If flow context information cannot be retrieved or app for the given * If flow context information cannot be retrieved or app for the given
* app id cannot be found, HTTP 404(Not Found) is returned. * app id cannot be found, HTTP 404(Not Found) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1361,16 +1322,16 @@ public TimelineEntity getApp(
* @param userId User id which should match for the app(Optional query param). * @param userId User id which should match for the app(Optional query param).
* @param fields Specifies which fields of the app entity object to retrieve, * @param fields Specifies which fields of the app entity object to retrieve,
* see {@link Field}. All fields will be retrieved if fields=ALL. If not * see {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
* app id and app created time is returned(Optional query param). * app id and app created time is returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing a * @return If successful, a HTTP 200(OK) response having a JSON representing a
* {@link TimelineEntity} instance is returned. * <cite>TimelineEntity</cite> instance is returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* If flow context information cannot be retrieved or app for the given * If flow context information cannot be retrieved or app for the given
* app id cannot be found, HTTP 404(Not Found) is returned. * app id cannot be found, HTTP 404(Not Found) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1399,14 +1360,11 @@ public TimelineEntity getApp(
TimelineEntity entity = null; TimelineEntity entity = null;
try { try {
entity = timelineReaderManager.getEntity( entity = timelineReaderManager.getEntity(
TimelineReaderWebServicesUtils.parseStr(userId), TimelineReaderWebServicesUtils.createTimelineReaderContext(
TimelineReaderWebServicesUtils.parseStr(clusterId), clusterId, userId, flowName, flowRunId, appId,
TimelineReaderWebServicesUtils.parseStr(flowName), TimelineEntityType.YARN_APPLICATION.toString(), null),
TimelineReaderWebServicesUtils.parseLongStr(flowRunId), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
TimelineReaderWebServicesUtils.parseStr(appId), null, null, fields));
TimelineEntityType.YARN_APPLICATION.toString(), null,
TimelineReaderWebServicesUtils.parseFieldsStr(
fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, "flowrunid"); handleException(e, url, startTime, "flowrunid");
} }
@ -1459,15 +1417,15 @@ public TimelineEntity getApp(
* events. This is represented as eventfilters=eventid1, eventid2... * events. This is represented as eventfilters=eventid1, eventid2...
* @param fields Specifies which fields of the app entity object to retrieve, * @param fields Specifies which fields of the app entity object to retrieve,
* see {@link Field}. All fields will be retrieved if fields=ALL. If not * see {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
* app id and app created time is returned(Optional query param). * app id and app created time is returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing * @return If successful, a HTTP 200(OK) response having a JSON representing
* a set of {@link TimelineEntity} instances representing apps is * a set of <cite>TimelineEntity</cite> instances representing apps is
* returned. * returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request or UID is incorrect, * If any problem occurs in parsing request or UID is incorrect,
* HTTP 400(Bad Request) is returned. * HTTP 400(Bad Request) is returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1505,30 +1463,13 @@ public Set<TimelineEntity> getFlowRunApps(
if (context == null) { if (context == null) {
throw new BadRequestException("Incorrect UID " + uId); throw new BadRequestException("Incorrect UID " + uId);
} }
entities = timelineReaderManager.getEntities( context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString());
TimelineReaderWebServicesUtils.parseStr(context.getUserId()), entities = timelineReaderManager.getEntities(context,
TimelineReaderWebServicesUtils.parseStr(context.getClusterId()), TimelineReaderWebServicesUtils.createTimelineEntityFilters(
TimelineReaderWebServicesUtils.parseStr(context.getFlowName()), limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
context.getFlowRunId(), infofilters, conffilters, metricfilters, eventfilters),
TimelineReaderWebServicesUtils.parseStr(context.getAppId()), TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
TimelineEntityType.YARN_APPLICATION.toString(), null, null, fields));
TimelineReaderWebServicesUtils.parseLongStr(limit),
TimelineReaderWebServicesUtils.parseLongStr(createdTimeStart),
TimelineReaderWebServicesUtils.parseLongStr(createdTimeEnd),
TimelineReaderWebServicesUtils.parseKeyStrValuesStr(
relatesTo, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValuesStr(
isRelatedTo, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValueObj(
infofilters, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseKeyStrValueStr(
conffilters, COMMA_DELIMITER, COLON_DELIMITER),
TimelineReaderWebServicesUtils.parseValuesStr(
metricfilters, COMMA_DELIMITER),
TimelineReaderWebServicesUtils.parseValuesStr(
eventfilters, COMMA_DELIMITER),
TimelineReaderWebServicesUtils.parseFieldsStr(
fields, COMMA_DELIMITER));
} catch (Exception e) { } catch (Exception e) {
handleException(e, url, startTime, handleException(e, url, startTime,
"createdTime start/end or limit or flowrunid"); "createdTime start/end or limit or flowrunid");
@ -1582,15 +1523,15 @@ public Set<TimelineEntity> getFlowRunApps(
* events. This is represented as eventfilters=eventid1, eventid2... * events. This is represented as eventfilters=eventid1, eventid2...
* @param fields Specifies which fields of the app entity object to retrieve, * @param fields Specifies which fields of the app entity object to retrieve,
* see {@link Field}. All fields will be retrieved if fields=ALL. If not * see {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
* app id and app created time is returned(Optional query param). * app id and app created time is returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing * @return If successful, a HTTP 200(OK) response having a JSON representing
* a set of {@link TimelineEntity} instances representing apps is * a set of <cite>TimelineEntity</cite> instances representing apps is
* returned. * returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1661,15 +1602,15 @@ public Set<TimelineEntity> getFlowRunApps(
* events. This is represented as eventfilters=eventid1, eventid2... * events. This is represented as eventfilters=eventid1, eventid2...
* @param fields Specifies which fields of the app entity object to retrieve, * @param fields Specifies which fields of the app entity object to retrieve,
* see {@link Field}. All fields will be retrieved if fields=ALL. If not * see {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
* app id and app created time is returned(Optional query param). * app id and app created time is returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing * @return If successful, a HTTP 200(OK) response having a JSON representing
* a set of {@link TimelineEntity} instances representing apps is * a set of <cite>TimelineEntity</cite> instances representing apps is
* returned. * returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1739,15 +1680,15 @@ public Set<TimelineEntity> getFlowRunApps(
* events. This is represented as eventfilters=eventid1, eventid2... * events. This is represented as eventfilters=eventid1, eventid2...
* @param fields Specifies which fields of the app entity object to retrieve, * @param fields Specifies which fields of the app entity object to retrieve,
* see {@link Field}. All fields will be retrieved if fields=ALL. If not * see {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
* app id and app created time is returned(Optional query param). * app id and app created time is returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing * @return If successful, a HTTP 200(OK) response having a JSON representing
* a set of {@link TimelineEntity} instances representing apps is * a set of <cite>TimelineEntity</cite> instances representing apps is
* returned. * returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */
@ -1815,15 +1756,15 @@ public Set<TimelineEntity> getFlowApps(
* events. This is represented as eventfilters=eventid1, eventid2... * events. This is represented as eventfilters=eventid1, eventid2...
* @param fields Specifies which fields of the app entity object to retrieve, * @param fields Specifies which fields of the app entity object to retrieve,
* see {@link Field}. All fields will be retrieved if fields=ALL. If not * see {@link Field}. All fields will be retrieved if fields=ALL. If not
* specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
* app id and app created time is returned(Optional query param). * app id and app created time is returned(Optional query param).
* *
* @return If successful, a HTTP 200(OK) response having a JSON representing * @return If successful, a HTTP 200(OK) response having a JSON representing
* a set of {@link TimelineEntity} instances representing apps is * a set of <cite>TimelineEntity</cite> instances representing apps is
* returned. * returned.<br>
* On failures, * On failures,<br>
* If any problem occurs in parsing request, HTTP 400(Bad Request) is * If any problem occurs in parsing request, HTTP 400(Bad Request) is
* returned. * returned.<br>
* For all other errors while retrieving data, HTTP 500(Internal Server * For all other errors while retrieving data, HTTP 500(Internal Server
* Error) is returned. * Error) is returned.
*/ */

View File

@ -35,9 +35,77 @@
* Set of utility methods to be used by timeline reader web services. * Set of utility methods to be used by timeline reader web services.
*/ */
final class TimelineReaderWebServicesUtils { final class TimelineReaderWebServicesUtils {
private static final String COMMA_DELIMITER = ",";
private static final String COLON_DELIMITER = ":";
private TimelineReaderWebServicesUtils() { private TimelineReaderWebServicesUtils() {
} }
/**
* Parse the passed context information represented as strings and convert
* into a {@link TimelineReaderContext} object.
* @param clusterId
* @param userId
* @param flowName
* @param flowRunId
* @param appId
* @param entityType
* @param entityId
* @return a {@link TimelineReaderContext} object.
* @throws Exception
*/
static TimelineReaderContext createTimelineReaderContext(String clusterId,
String userId, String flowName, String flowRunId, String appId,
String entityType, String entityId) throws Exception {
return new TimelineReaderContext(parseStr(clusterId), parseStr(userId),
parseStr(flowName), parseLongStr(flowRunId), parseStr(appId),
parseStr(entityType), parseStr(entityId));
}
/**
* Parse the passed filters represented as strings and convert them into a
* {@link TimelineEntityFilters} object.
* @param limit
* @param createdTimeStart
* @param createdTimeEnd
* @param relatesTo
* @param isRelatedTo
* @param infofilters
* @param conffilters
* @param metricfilters
* @param eventfilters
* @return a {@link TimelineEntityFilters} object.
* @throws Exception
*/
static TimelineEntityFilters createTimelineEntityFilters(String limit,
String createdTimeStart, String createdTimeEnd, String relatesTo,
String isRelatedTo, String infofilters, String conffilters,
String metricfilters, String eventfilters) throws Exception {
return new TimelineEntityFilters(parseLongStr(limit),
parseLongStr(createdTimeStart), parseLongStr(createdTimeEnd),
parseKeyStrValuesStr(relatesTo, COMMA_DELIMITER, COLON_DELIMITER),
parseKeyStrValuesStr(isRelatedTo, COMMA_DELIMITER, COLON_DELIMITER),
parseKeyStrValueObj(infofilters, COMMA_DELIMITER, COLON_DELIMITER),
parseKeyStrValueStr(conffilters, COMMA_DELIMITER, COLON_DELIMITER),
parseValuesStr(metricfilters, COMMA_DELIMITER),
parseValuesStr(eventfilters, COMMA_DELIMITER));
}
/**
* Parse the passed fields represented as strings and convert them into a
* {@link TimelineDataToRetrieve} object.
* @param confs
* @param metrics
* @param fields
* @return a {@link TimelineDataToRetrieve} object.
* @throws Exception
*/
static TimelineDataToRetrieve createTimelineDataToRetrieve(String confs,
String metrics, String fields) throws Exception {
return new TimelineDataToRetrieve(
null, null, parseFieldsStr(fields, COMMA_DELIMITER));
}
/** /**
* Parse a delimited string and convert it into a set of strings. For * Parse a delimited string and convert it into a set of strings. For
* instance, if delimiter is ",", then the string should be represented as * instance, if delimiter is ",", then the string should be represented as

View File

@ -44,7 +44,9 @@
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils; import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.JsonGenerationException;
@ -264,22 +266,8 @@ private static TimelineEntity readEntityFromFile(BufferedReader reader)
} }
private Set<TimelineEntity> getEntities(File dir, String entityType, private Set<TimelineEntity> getEntities(File dir, String entityType,
Long limit, Long createdTimeBegin, Long createdTimeEnd, TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve)
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo, throws IOException {
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve) throws IOException {
if (limit == null || limit <= 0) {
limit = DEFAULT_LIMIT;
}
if (createdTimeBegin == null || createdTimeBegin <= 0) {
createdTimeBegin = 0L;
}
if (createdTimeEnd == null || createdTimeEnd <= 0) {
createdTimeEnd = Long.MAX_VALUE;
}
// First sort the selected entities based on created/start time. // First sort the selected entities based on created/start time.
Map<Long, Set<TimelineEntity>> sortedEntities = Map<Long, Set<TimelineEntity>> sortedEntities =
new TreeMap<>( new TreeMap<>(
@ -303,41 +291,48 @@ public int compare(Long l1, Long l2) {
if (!entity.getType().equals(entityType)) { if (!entity.getType().equals(entityType)) {
continue; continue;
} }
if (!isTimeInRange(entity.getCreatedTime(), createdTimeBegin, if (!isTimeInRange(entity.getCreatedTime(),
createdTimeEnd)) { filters.getCreatedTimeBegin(), filters.getCreatedTimeEnd())) {
continue; continue;
} }
if (relatesTo != null && !relatesTo.isEmpty() && if (filters.getRelatesTo() != null &&
!TimelineStorageUtils !filters.getRelatesTo().isEmpty() &&
.matchRelations(entity.getRelatesToEntities(), relatesTo)) { !TimelineStorageUtils.matchRelations(
entity.getRelatesToEntities(), filters.getRelatesTo())) {
continue; continue;
} }
if (isRelatedTo != null && !isRelatedTo.isEmpty() && if (filters.getIsRelatedTo() != null &&
!TimelineStorageUtils !filters.getIsRelatedTo().isEmpty() &&
.matchRelations(entity.getIsRelatedToEntities(), isRelatedTo)) { !TimelineStorageUtils.matchRelations(
entity.getIsRelatedToEntities(), filters.getIsRelatedTo())) {
continue; continue;
} }
if (infoFilters != null && !infoFilters.isEmpty() && if (filters.getInfoFilters() != null &&
!TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) { !filters.getInfoFilters().isEmpty() &&
continue;
}
if (configFilters != null && !configFilters.isEmpty() &&
!TimelineStorageUtils.matchFilters( !TimelineStorageUtils.matchFilters(
entity.getConfigs(), configFilters)) { entity.getInfo(), filters.getInfoFilters())) {
continue; continue;
} }
if (metricFilters != null && !metricFilters.isEmpty() && if (filters.getConfigFilters() != null &&
!filters.getConfigFilters().isEmpty() &&
!TimelineStorageUtils.matchFilters(
entity.getConfigs(), filters.getConfigFilters())) {
continue;
}
if (filters.getMetricFilters() != null &&
!filters.getMetricFilters().isEmpty() &&
!TimelineStorageUtils.matchMetricFilters( !TimelineStorageUtils.matchMetricFilters(
entity.getMetrics(), metricFilters)) { entity.getMetrics(), filters.getMetricFilters())) {
continue; continue;
} }
if (eventFilters != null && !eventFilters.isEmpty() && if (filters.getEventFilters() != null &&
!filters.getEventFilters().isEmpty() &&
!TimelineStorageUtils.matchEventFilters( !TimelineStorageUtils.matchEventFilters(
entity.getEvents(), eventFilters)) { entity.getEvents(), filters.getEventFilters())) {
continue; continue;
} }
TimelineEntity entityToBeReturned = TimelineEntity entityToBeReturned = createEntityToBeReturned(
createEntityToBeReturned(entity, fieldsToRetrieve); entity, dataToRetrieve.getFieldsToRetrieve());
Set<TimelineEntity> entitiesCreatedAtSameTime = Set<TimelineEntity> entitiesCreatedAtSameTime =
sortedEntities.get(entityToBeReturned.getCreatedTime()); sortedEntities.get(entityToBeReturned.getCreatedTime());
if (entitiesCreatedAtSameTime == null) { if (entitiesCreatedAtSameTime == null) {
@ -355,7 +350,7 @@ public int compare(Long l1, Long l2) {
for (TimelineEntity entity : entitySet) { for (TimelineEntity entity : entitySet) {
entities.add(entity); entities.add(entity);
++entitiesAdded; ++entitiesAdded;
if (entitiesAdded >= limit) { if (entitiesAdded >= filters.getLimit()) {
return entities; return entities;
} }
} }
@ -371,45 +366,40 @@ public void serviceInit(Configuration conf) throws Exception {
} }
@Override @Override
public TimelineEntity getEntity(String userId, String clusterId, public TimelineEntity getEntity(TimelineReaderContext context,
String flowName, Long flowRunId, String appId, String entityType, TimelineDataToRetrieve dataToRetrieve) throws IOException {
String entityId, TimelineFilterList confsToRetrieve, String flowRunPath = getFlowRunPath(context.getUserId(),
TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) context.getClusterId(), context.getFlowName(), context.getFlowRunId(),
throws IOException { context.getAppId());
String flowRunPath = getFlowRunPath(userId, clusterId, flowName,
flowRunId, appId);
File dir = new File(new File(rootPath, ENTITIES_DIR), File dir = new File(new File(rootPath, ENTITIES_DIR),
clusterId + "/" + flowRunPath + "/" + appId + "/" + entityType); context.getClusterId() + "/" + flowRunPath + "/" + context.getAppId() +
File entityFile = "/" + context.getEntityType());
new File(dir, entityId + TIMELINE_SERVICE_STORAGE_EXTENSION); File entityFile = new File(
dir, context.getEntityId() + TIMELINE_SERVICE_STORAGE_EXTENSION);
try (BufferedReader reader = try (BufferedReader reader =
new BufferedReader(new InputStreamReader( new BufferedReader(new InputStreamReader(
new FileInputStream(entityFile), Charset.forName("UTF-8")))) { new FileInputStream(entityFile), Charset.forName("UTF-8")))) {
TimelineEntity entity = readEntityFromFile(reader); TimelineEntity entity = readEntityFromFile(reader);
return createEntityToBeReturned(entity, fieldsToRetrieve); return createEntityToBeReturned(
entity, dataToRetrieve.getFieldsToRetrieve());
} catch (FileNotFoundException e) { } catch (FileNotFoundException e) {
LOG.info("Cannot find entity {id:" + entityId + " , type:" + entityType + LOG.info("Cannot find entity {id:" + context.getEntityId() + " , type:" +
"}. Will send HTTP 404 in response."); context.getEntityType() + "}. Will send HTTP 404 in response.");
return null; return null;
} }
} }
@Override @Override
public Set<TimelineEntity> getEntities(String userId, String clusterId, public Set<TimelineEntity> getEntities(TimelineReaderContext context,
String flowName, Long flowRunId, String appId, String entityType, TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve)
Long limit, Long createdTimeBegin, Long createdTimeEnd, throws IOException {
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo, String flowRunPath = getFlowRunPath(context.getUserId(),
Map<String, Object> infoFilters, Map<String, String> configFilters, context.getClusterId(), context.getFlowName(), context.getFlowRunId(),
Set<String> metricFilters, Set<String> eventFilters, context.getAppId());
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve) throws IOException {
String flowRunPath =
getFlowRunPath(userId, clusterId, flowName, flowRunId, appId);
File dir = File dir =
new File(new File(rootPath, ENTITIES_DIR), new File(new File(rootPath, ENTITIES_DIR),
clusterId + "/" + flowRunPath + "/" + appId + "/" + entityType); context.getClusterId() + "/" + flowRunPath + "/" +
return getEntities(dir, entityType, limit, createdTimeBegin, createdTimeEnd, context.getAppId() + "/" + context.getEntityType());
relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters, return getEntities(dir, context.getEntityType(), filters, dataToRetrieve);
eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
} }
} }

View File

@ -19,8 +19,6 @@
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet;
import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -31,7 +29,9 @@
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader; import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader;
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory; import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory;
@ -65,33 +65,21 @@ protected void serviceStop() throws Exception {
} }
@Override @Override
public TimelineEntity getEntity(String userId, String clusterId, public TimelineEntity getEntity(TimelineReaderContext context,
String flowName, Long flowRunId, String appId, String entityType, TimelineDataToRetrieve dataToRetrieve) throws IOException {
String entityId, TimelineFilterList confsToRetrieve,
TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve)
throws IOException {
TimelineEntityReader reader = TimelineEntityReader reader =
TimelineEntityReaderFactory.createSingleEntityReader(userId, clusterId, TimelineEntityReaderFactory.createSingleEntityReader(context,
flowName, flowRunId, appId, entityType, entityId, confsToRetrieve, dataToRetrieve);
metricsToRetrieve, fieldsToRetrieve);
return reader.readEntity(hbaseConf, conn); return reader.readEntity(hbaseConf, conn);
} }
@Override @Override
public Set<TimelineEntity> getEntities(String userId, String clusterId, public Set<TimelineEntity> getEntities(TimelineReaderContext context,
String flowName, Long flowRunId, String appId, String entityType, TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve)
Long limit, Long createdTimeBegin, Long createdTimeEnd, throws IOException {
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve) throws IOException {
TimelineEntityReader reader = TimelineEntityReader reader =
TimelineEntityReaderFactory.createMultipleEntitiesReader(userId, TimelineEntityReaderFactory.createMultipleEntitiesReader(context,
clusterId, flowName, flowRunId, appId, entityType, limit, filters, dataToRetrieve);
createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo,
infoFilters, configFilters, metricFilters, eventFilters,
confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
return reader.readEntities(hbaseConf, conn); return reader.readEntities(hbaseConf, conn);
} }
} }

View File

@ -20,28 +20,20 @@
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet;
import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.service.Service; import org.apache.hadoop.service.Service;
import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
/** ATSv2 reader interface. */ /** ATSv2 reader interface. */
@Private @Private
@Unstable @Unstable
public interface TimelineReader extends Service { public interface TimelineReader extends Service {
/**
* Default limit for {@link #getEntities}.
*/
long DEFAULT_LIMIT = 100;
/** /**
* Possible fields to retrieve for {@link #getEntities} and * Possible fields to retrieve for {@link #getEntities} and
* {@link #getEntity}. * {@link #getEntity}.
@ -57,55 +49,61 @@ public enum Field {
} }
/** /**
* <p>The API to fetch the single entity given the entity identifier in the * <p>The API to fetch the single entity given the identifier(depending on
* scope of the given context.</p> * the entity type) in the scope of the given context.</p>
* * @param context Context which defines the scope in which query has to be
* @param userId * made. Use getters of {@link TimelineReaderContext} to fetch context
* Context user Id(optional). * fields. Context contains the following :<br>
* @param clusterId * <ul>
* Context cluster Id(mandatory). * <li><b>entityType</b> - Entity type(mandatory).</li>
* @param flowName * <li><b>clusterId</b> - Identifies the cluster(mandatory).</li>
* Context flow Id (optional). * <li><b>userId</b> - Identifies the user.</li>
* @param flowRunId * <li><b>flowName</b> - Context flow name.</li>
* Context flow run Id (optional). * <li><b>flowRunId</b> - Context flow run id.</li>
* @param appId * <li><b>appId</b> - Context app id.</li>
* Context app Id (mandatory) * <li><b>entityId</b> - Entity id.</li>
* @param entityType * </ul>
* Entity type (mandatory) * Fields in context which are mandatory depends on entity type. Entity
* @param entityId * type is always mandatory. In addition to entity type, below is the list
* Entity Id (mandatory) * of context fields which are mandatory, based on entity type.<br>
* @param confsToRetrieve * <ul>
* Used for deciding which configs to return in response. This is * <li>If entity type is YARN_FLOW_RUN (i.e. query to fetch a specific flow
* represented as a {@link TimelineFilterList} object containing * run), clusterId, userId, flowName and flowRunId are mandatory.</li>
* {@link TimelinePrefixFilter} objects. These can either be exact config * <li>If entity type is YARN_APPLICATION (i.e. query to fetch a specific
* keys' or prefixes which are then compared against config keys' to decide * app), query is within the scope of clusterId, userId, flowName,
* configs to return in response. * flowRunId and appId. But out of this, only clusterId and appId are
* @param metricsToRetrieve * mandatory. If only clusterId and appId are supplied, backend storage
* Used for deciding which metrics to return in response. This is * must fetch the flow context information i.e. userId, flowName and
* represented as a {@link TimelineFilterList} object containing * flowRunId first and based on that, fetch the app. If flow context
* {@link TimelinePrefixFilter} objects. These can either be exact metric * information is also given, app can be directly fetched.
* ids' or prefixes which are then compared against metric ids' to decide * </li>
* metrics to return in response. * <li>For other entity types (i.e. query to fetch generic entity), query
* @param fieldsToRetrieve * is within the scope of clusterId, userId, flowName, flowRunId, appId,
* Specifies which fields of the entity object to retrieve(optional), see * entityType and entityId. But out of this, only clusterId, appId,
* {@link Field}. If null, retrieves 4 fields namely entity id, * entityType and entityId are mandatory. If flow context information is
* entity type and entity created time. All fields will be returned if * not supplied, backend storage must fetch the flow context information
* {@link Field#ALL} is specified. * i.e. userId, flowName and flowRunId first and based on that, fetch the
* @return a {@link TimelineEntity} instance or null. The entity will * entity. If flow context information is also given, entity can be
* contain the metadata plus the given fields to retrieve. * directly queried.
* </li>
* </ul>
* @param dataToRetrieve Specifies which data to retrieve for the entity. Use
* getters of TimelineDataToRetrieve class to fetch dataToRetrieve
* fields. All the dataToRetrieve fields are optional. Refer to
* {@link TimelineDataToRetrieve} for details.
* @return A <cite>TimelineEntity</cite> instance or null. The entity will
* contain the metadata plus the given fields to retrieve.<br>
* If entityType is YARN_FLOW_RUN, entity returned is of type * If entityType is YARN_FLOW_RUN, entity returned is of type
* {@link FlowRunEntity}. * <cite>FlowRunEntity</cite>.<br>
* For all other entity types, entity returned is of type * For all other entity types, entity returned is of type
* {@link TimelineEntity}. * <cite>TimelineEntity</cite>.
* @throws IOException * @throws IOException
*/ */
TimelineEntity getEntity(String userId, String clusterId, String flowName, TimelineEntity getEntity(TimelineReaderContext context,
Long flowRunId, String appId, String entityType, String entityId, TimelineDataToRetrieve dataToRetrieve) throws IOException;
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve) throws IOException;
/** /**
* <p>The API to search for a set of entities of the given the entity type in * <p>The API to search for a set of entities of the given entity type in
* the scope of the given context which matches the given predicates. The * the scope of the given context which matches the given predicates. The
* predicates include the created time window, limit to number of entities to * predicates include the created time window, limit to number of entities to
* be returned, and the entities can be filtered by checking whether they * be returned, and the entities can be filtered by checking whether they
@ -115,84 +113,66 @@ TimelineEntity getEntity(String userId, String clusterId, String flowName,
* related to other entities. For those parameters which have multiple * related to other entities. For those parameters which have multiple
* entries, the qualified entity needs to meet all or them.</p> * entries, the qualified entity needs to meet all or them.</p>
* *
* @param userId * @param context Context which defines the scope in which query has to be
* Context user Id(optional). * made. Use getters of {@link TimelineReaderContext} to fetch context
* @param clusterId * fields. Context contains the following :<br>
* Context cluster Id(mandatory). * <ul>
* @param flowName * <li><b>entityType</b> - Entity type(mandatory).</li>
* Context flow Id (optional). * <li><b>clusterId</b> - Identifies the cluster(mandatory).</li>
* @param flowRunId * <li><b>userId</b> - Identifies the user.</li>
* Context flow run Id (optional). * <li><b>flowName</b> - Context flow name.</li>
* @param appId * <li><b>flowRunId</b> - Context flow run id.</li>
* Context app Id (mandatory) * <li><b>appId</b> - Context app id.</li>
* @param entityType * </ul>
* Entity type (mandatory) * Although entityId is also part of context, it has no meaning for
* @param limit * getEntities.<br>
* A limit on the number of entities to return (optional). If null or <=0, * Fields in context which are mandatory depends on entity type. Entity
* defaults to {@link #DEFAULT_LIMIT}. * type is always mandatory. In addition to entity type, below is the list
* @param createdTimeBegin * of context fields which are mandatory, based on entity type.<br>
* Matched entities should not be created before this timestamp (optional). * <ul>
* If null or <=0, defaults to 0. * <li>If entity type is YARN_FLOW_ACTIVITY (i.e. query to fetch flows),
* @param createdTimeEnd * only clusterId is mandatory.
* Matched entities should not be created after this timestamp (optional). * </li>
* If null or <=0, defaults to {@link Long#MAX_VALUE}. * <li>If entity type is YARN_FLOW_RUN (i.e. query to fetch flow runs),
* @param relatesTo * clusterId, userId and flowName are mandatory.</li>
* Matched entities should relate to given entities (optional). * <li>If entity type is YARN_APPLICATION (i.e. query to fetch apps), we
* @param isRelatedTo * can either get all apps within the context of flow name or within the
* Matched entities should be related to given entities (optional). * context of flow run. If apps are queried within the scope of flow name,
* @param infoFilters * clusterId, userId and flowName are supplied. If they are queried within
* Matched entities should have exact matches to the given info represented * the scope of flow run, clusterId, userId, flowName and flowRunId are
* as key-value pairs (optional). If null or empty, the filter is not * supplied.</li>
* applied. * <li>For other entity types (i.e. query to fetch generic entities), query
* @param configFilters * is within the scope of clusterId, userId, flowName, flowRunId, appId and
* Matched entities should have exact matches to the given configs * entityType. But out of this, only clusterId, appId and entityType are
* represented as key-value pairs (optional). If null or empty, the filter * mandatory. If flow context information is not supplied, backend storage
* is not applied. * must fetch the flow context information i.e. userId, flowName and
* @param metricFilters * flowRunId first and based on that, fetch the entities. If flow context
* Matched entities should contain the given metrics (optional). If null * information is also given, entities can be directly queried.
* or empty, the filter is not applied. * </li>
* @param eventFilters * </ul>
* Matched entities should contain the given events (optional). If null * @param filters Specifies filters which restrict the number of entities
* or empty, the filter is not applied. * to return. Use getters of TimelineEntityFilters class to fetch
* @param confsToRetrieve * various filters. All the filters are optional. Refer to
* Used for deciding which configs to return in response. This is * {@link TimelineEntityFilters} for details.
* represented as a {@link TimelineFilterList} object containing * @param dataToRetrieve Specifies which data to retrieve for each entity. Use
* {@link TimelinePrefixFilter} objects. These can either be exact config * getters of TimelineDataToRetrieve class to fetch dataToRetrieve
* keys' or prefixes which are then compared against config keys' to decide * fields. All the dataToRetrieve fields are optional. Refer to
* configs(inside entities) to return in response. This should not be * {@link TimelineDataToRetrieve} for details.
* confused with configFilters which is used to decide which entities to * @return A set of <cite>TimelineEntity</cite> instances of the given entity
* return instead. * type in the given context scope which matches the given predicates
* @param metricsToRetrieve
* Used for deciding which metrics to return in response. This is
* represented as a {@link TimelineFilterList} object containing
* {@link TimelinePrefixFilter} objects. These can either be exact metric
* ids' or prefixes which are then compared against metric ids' to decide
* metrics(inside entities) to return in response. This should not be
* confused with metricFilters which is used to decide which entities to
* return instead.
* @param fieldsToRetrieve
* Specifies which fields of the entity object to retrieve(optional), see
* {@link Field}. If null, retrieves 4 fields namely entity id,
* entity type and entity created time. All fields will be returned if
* {@link Field#ALL} is specified.
* @return A set of {@link TimelineEntity} instances of the given entity type
* in the given context scope which matches the given predicates
* ordered by created time, descending. Each entity will only contain the * ordered by created time, descending. Each entity will only contain the
* metadata(id, type and created time) plus the given fields to retrieve. * metadata(id, type and created time) plus the given fields to retrieve.
* <br>
* If entityType is YARN_FLOW_ACTIVITY, entities returned are of type * If entityType is YARN_FLOW_ACTIVITY, entities returned are of type
* {@link FlowActivityEntity}. * <cite>FlowActivityEntity</cite>.<br>
* If entityType is YARN_FLOW_RUN, entities returned are of type * If entityType is YARN_FLOW_RUN, entities returned are of type
* {@link FlowRunEntity}. * <cite>FlowRunEntity</cite>.<br>
* For all other entity types, entities returned are of type * For all other entity types, entities returned are of type
* {@link TimelineEntity}. * <cite>TimelineEntity</cite>.
* @throws IOException * @throws IOException
*/ */
Set<TimelineEntity> getEntities(String userId, String clusterId, Set<TimelineEntity> getEntities(
String flowName, Long flowRunId, String appId, String entityType, TimelineReaderContext context,
Long limit, Long createdTimeBegin, Long createdTimeEnd, TimelineEntityFilters filters,
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo, TimelineDataToRetrieve dataToRetrieve) throws IOException;
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve) throws IOException;
} }

View File

@ -19,8 +19,6 @@
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
@ -38,9 +36,10 @@
import org.apache.hadoop.hbase.filter.QualifierFilter; import org.apache.hadoop.hbase.filter.QualifierFilter;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; 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.TimelineEntityType;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn; import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily; import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily;
@ -60,26 +59,14 @@ class ApplicationEntityReader extends GenericEntityReader {
private static final ApplicationTable APPLICATION_TABLE = private static final ApplicationTable APPLICATION_TABLE =
new ApplicationTable(); new ApplicationTable();
public ApplicationEntityReader(String userId, String clusterId, public ApplicationEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
Long limit, Long createdTimeBegin, Long createdTimeEnd, super(ctxt, entityFilters, toRetrieve, true);
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve) {
super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo, infoFilters,
configFilters, metricFilters, eventFilters, confsToRetrieve,
metricsToRetrieve, fieldsToRetrieve, true);
} }
public ApplicationEntityReader(String userId, String clusterId, public ApplicationEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineDataToRetrieve toRetrieve) {
String entityId, TimelineFilterList confsToRetrieve, super(ctxt, toRetrieve);
TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
} }
/** /**
@ -92,12 +79,13 @@ protected BaseTable<?> getTable() {
@Override @Override
protected FilterList constructFilterListBasedOnFields() { protected FilterList constructFilterListBasedOnFields() {
FilterList list = new FilterList(Operator.MUST_PASS_ONE); FilterList list = new FilterList(Operator.MUST_PASS_ONE);
TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
// Fetch all the columns. // Fetch all the columns.
if (fieldsToRetrieve.contains(Field.ALL) && if (dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(confsToRetrieve == null || (dataToRetrieve.getConfsToRetrieve() == null ||
confsToRetrieve.getFilterList().isEmpty()) && dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) &&
(metricsToRetrieve == null || (dataToRetrieve.getMetricsToRetrieve() == null ||
metricsToRetrieve.getFilterList().isEmpty())) { dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) {
return list; return list;
} }
FilterList infoColFamilyList = new FilterList(); FilterList infoColFamilyList = new FilterList();
@ -107,61 +95,70 @@ protected FilterList constructFilterListBasedOnFields() {
new BinaryComparator(ApplicationColumnFamily.INFO.getBytes())); new BinaryComparator(ApplicationColumnFamily.INFO.getBytes()));
infoColFamilyList.addFilter(infoColumnFamily); infoColFamilyList.addFilter(infoColumnFamily);
// Events not required. // Events not required.
if (!fieldsToRetrieve.contains(Field.EVENTS) && TimelineEntityFilters filters = getFilters();
!fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) { if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.EVENTS) &&
!dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(singleEntityRead || filters.getEventFilters() == null)) {
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL, new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator( new BinaryPrefixComparator(
ApplicationColumnPrefix.EVENT.getColumnPrefixBytes("")))); ApplicationColumnPrefix.EVENT.getColumnPrefixBytes(""))));
} }
// info not required. // info not required.
if (!fieldsToRetrieve.contains(Field.INFO) && if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.INFO) &&
!fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) { !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(singleEntityRead || filters.getInfoFilters() == null)) {
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL, new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator( new BinaryPrefixComparator(
ApplicationColumnPrefix.INFO.getColumnPrefixBytes("")))); ApplicationColumnPrefix.INFO.getColumnPrefixBytes(""))));
} }
// is releated to not required. // is releated to not required.
if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) && if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.IS_RELATED_TO) &&
!fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) { !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(singleEntityRead || filters.getIsRelatedTo() == null)) {
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL, new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator( new BinaryPrefixComparator(
ApplicationColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes("")))); ApplicationColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
} }
// relates to not required. // relates to not required.
if (!fieldsToRetrieve.contains(Field.RELATES_TO) && if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.RELATES_TO) &&
!fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) { !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(singleEntityRead || filters.getRelatesTo() == null)) {
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL, new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator( new BinaryPrefixComparator(
ApplicationColumnPrefix.RELATES_TO.getColumnPrefixBytes("")))); ApplicationColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
} }
list.addFilter(infoColFamilyList); list.addFilter(infoColFamilyList);
if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) || if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS) ||
(confsToRetrieve != null && (!singleEntityRead && filters.getConfigFilters() != null)) ||
!confsToRetrieve.getFilterList().isEmpty())) { (dataToRetrieve.getConfsToRetrieve() != null &&
!dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty())) {
FilterList filterCfg = FilterList filterCfg =
new FilterList(new FamilyFilter(CompareOp.EQUAL, new FilterList(new FamilyFilter(CompareOp.EQUAL,
new BinaryComparator(ApplicationColumnFamily.CONFIGS.getBytes()))); new BinaryComparator(ApplicationColumnFamily.CONFIGS.getBytes())));
if (confsToRetrieve != null && if (dataToRetrieve.getConfsToRetrieve() != null &&
!confsToRetrieve.getFilterList().isEmpty()) { !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) {
filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList( filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
ApplicationColumnPrefix.CONFIG, confsToRetrieve)); ApplicationColumnPrefix.CONFIG,
dataToRetrieve.getConfsToRetrieve()));
} }
list.addFilter(filterCfg); list.addFilter(filterCfg);
} }
if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) || if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) ||
(metricsToRetrieve != null && (!singleEntityRead && filters.getMetricFilters() != null)) ||
!metricsToRetrieve.getFilterList().isEmpty())) { (dataToRetrieve.getMetricsToRetrieve() != null &&
!dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) {
FilterList filterMetrics = FilterList filterMetrics =
new FilterList(new FamilyFilter(CompareOp.EQUAL, new FilterList(new FamilyFilter(CompareOp.EQUAL,
new BinaryComparator(ApplicationColumnFamily.METRICS.getBytes()))); new BinaryComparator(ApplicationColumnFamily.METRICS.getBytes())));
if (metricsToRetrieve != null && if (dataToRetrieve.getMetricsToRetrieve() != null &&
!metricsToRetrieve.getFilterList().isEmpty()) { !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) {
filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList( filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
ApplicationColumnPrefix.METRIC, metricsToRetrieve)); ApplicationColumnPrefix.METRIC,
dataToRetrieve.getMetricsToRetrieve()));
} }
list.addFilter(filterMetrics); list.addFilter(filterMetrics);
} }
@ -171,9 +168,10 @@ protected FilterList constructFilterListBasedOnFields() {
@Override @Override
protected Result getResult(Configuration hbaseConf, Connection conn, protected Result getResult(Configuration hbaseConf, Connection conn,
FilterList filterList) throws IOException { FilterList filterList) throws IOException {
TimelineReaderContext context = getContext();
byte[] rowKey = byte[] rowKey =
ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId, ApplicationRowKey.getRowKey(context.getClusterId(), context.getUserId(),
appId); context.getFlowName(), context.getFlowRunId(), context.getAppId());
Get get = new Get(rowKey); Get get = new Get(rowKey);
get.setMaxVersions(Integer.MAX_VALUE); get.setMaxVersions(Integer.MAX_VALUE);
if (filterList != null && !filterList.getFilters().isEmpty()) { if (filterList != null && !filterList.getFilters().isEmpty()) {
@ -184,66 +182,54 @@ protected Result getResult(Configuration hbaseConf, Connection conn,
@Override @Override
protected void validateParams() { protected void validateParams() {
Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null"); Preconditions.checkNotNull(getContext().getClusterId(),
Preconditions.checkNotNull(entityType, "entityType shouldn't be null"); "clusterId shouldn't be null");
Preconditions.checkNotNull(getContext().getEntityType(),
"entityType shouldn't be null");
if (singleEntityRead) { if (singleEntityRead) {
Preconditions.checkNotNull(appId, "appId shouldn't be null"); Preconditions.checkNotNull(getContext().getAppId(),
"appId shouldn't be null");
} else { } else {
Preconditions.checkNotNull(userId, "userId shouldn't be null"); Preconditions.checkNotNull(getContext().getUserId(),
Preconditions.checkNotNull(flowName, "flowName shouldn't be null"); "userId shouldn't be null");
Preconditions.checkNotNull(getContext().getFlowName(),
"flowName shouldn't be null");
} }
} }
@Override @Override
protected void augmentParams(Configuration hbaseConf, Connection conn) protected void augmentParams(Configuration hbaseConf, Connection conn)
throws IOException { throws IOException {
TimelineReaderContext context = getContext();
if (singleEntityRead) { if (singleEntityRead) {
if (flowName == null || flowRunId == null || userId == null) { if (context.getFlowName() == null || context.getFlowRunId() == null ||
FlowContext context = context.getUserId() == null) {
lookupFlowContext(clusterId, appId, hbaseConf, conn); FlowContext flowContext = lookupFlowContext(
flowName = context.flowName; context.getClusterId(), context.getAppId(), hbaseConf, conn);
flowRunId = context.flowRunId; context.setFlowName(flowContext.flowName);
userId = context.userId; context.setFlowRunId(flowContext.flowRunId);
} context.setUserId(flowContext.userId);
}
if (fieldsToRetrieve == null) {
fieldsToRetrieve = EnumSet.noneOf(Field.class);
}
if (!fieldsToRetrieve.contains(Field.CONFIGS) &&
confsToRetrieve != null && !confsToRetrieve.getFilterList().isEmpty()) {
fieldsToRetrieve.add(Field.CONFIGS);
}
if (!fieldsToRetrieve.contains(Field.METRICS) &&
metricsToRetrieve != null &&
!metricsToRetrieve.getFilterList().isEmpty()) {
fieldsToRetrieve.add(Field.METRICS);
}
if (!singleEntityRead) {
if (limit == null || limit < 0) {
limit = TimelineReader.DEFAULT_LIMIT;
}
if (createdTimeBegin == null) {
createdTimeBegin = DEFAULT_BEGIN_TIME;
}
if (createdTimeEnd == null) {
createdTimeEnd = DEFAULT_END_TIME;
} }
} }
getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve();
} }
@Override @Override
protected ResultScanner getResults(Configuration hbaseConf, protected ResultScanner getResults(Configuration hbaseConf,
Connection conn, FilterList filterList) throws IOException { Connection conn, FilterList filterList) throws IOException {
Scan scan = new Scan(); Scan scan = new Scan();
if (flowRunId != null) { TimelineReaderContext context = getContext();
if (context.getFlowRunId() != null) {
scan.setRowPrefixFilter(ApplicationRowKey. scan.setRowPrefixFilter(ApplicationRowKey.
getRowKeyPrefix(clusterId, userId, flowName, flowRunId)); getRowKeyPrefix(context.getClusterId(), context.getUserId(),
context.getFlowName(), context.getFlowRunId()));
} else { } else {
scan.setRowPrefixFilter(ApplicationRowKey. scan.setRowPrefixFilter(ApplicationRowKey.
getRowKeyPrefix(clusterId, userId, flowName)); getRowKeyPrefix(context.getClusterId(), context.getUserId(),
context.getFlowName()));
} }
FilterList newList = new FilterList(); FilterList newList = new FilterList();
newList.addFilter(new PageFilter(limit)); newList.addFilter(new PageFilter(getFilters().getLimit()));
if (filterList != null && !filterList.getFilters().isEmpty()) { if (filterList != null && !filterList.getFilters().isEmpty()) {
newList.addFilter(filterList); newList.addFilter(filterList);
} }
@ -261,23 +247,27 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
String entityId = ApplicationColumn.ID.readResult(result).toString(); String entityId = ApplicationColumn.ID.readResult(result).toString();
entity.setId(entityId); entity.setId(entityId);
TimelineEntityFilters filters = getFilters();
// fetch created time // fetch created time
Number createdTime = Number createdTime =
(Number)ApplicationColumn.CREATED_TIME.readResult(result); (Number)ApplicationColumn.CREATED_TIME.readResult(result);
entity.setCreatedTime(createdTime.longValue()); entity.setCreatedTime(createdTime.longValue());
if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin || if (!singleEntityRead &&
entity.getCreatedTime() > createdTimeEnd)) { (entity.getCreatedTime() < filters.getCreatedTimeBegin() ||
entity.getCreatedTime() > filters.getCreatedTimeEnd())) {
return null; return null;
} }
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
// fetch is related to entities // fetch is related to entities
boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0; boolean checkIsRelatedTo =
filters != null && filters.getIsRelatedTo() != null &&
filters.getIsRelatedTo().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) { fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO, readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
true); true);
if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations( if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations(
entity.getIsRelatedToEntities(), isRelatedTo)) { entity.getIsRelatedToEntities(), filters.getIsRelatedTo())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -287,13 +277,15 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch relates to entities // fetch relates to entities
boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0; boolean checkRelatesTo =
filters != null && filters.getRelatesTo() != null &&
filters.getRelatesTo().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) { fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO, readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
false); false);
if (checkRelatesTo && !TimelineStorageUtils.matchRelations( if (checkRelatesTo && !TimelineStorageUtils.matchRelations(
entity.getRelatesToEntities(), relatesTo)) { entity.getRelatesToEntities(), filters.getRelatesTo())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -303,12 +295,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch info // fetch info
boolean checkInfo = infoFilters != null && infoFilters.size() > 0; boolean checkInfo = filters != null && filters.getInfoFilters() != null &&
filters.getInfoFilters().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.INFO) || checkInfo) { fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false); readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
if (checkInfo && if (checkInfo &&
!TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) { !TimelineStorageUtils.matchFilters(
entity.getInfo(), filters.getInfoFilters())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -318,12 +312,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch configs // fetch configs
boolean checkConfigs = configFilters != null && configFilters.size() > 0; boolean checkConfigs =
filters != null && filters.getConfigFilters() != null &&
filters.getConfigFilters().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) { fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true); readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
if (checkConfigs && !TimelineStorageUtils.matchFilters( if (checkConfigs && !TimelineStorageUtils.matchFilters(
entity.getConfigs(), configFilters)) { entity.getConfigs(), filters.getConfigFilters())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -333,12 +329,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch events // fetch events
boolean checkEvents = eventFilters != null && eventFilters.size() > 0; boolean checkEvents =
filters != null && filters.getEventFilters() != null &&
filters.getEventFilters().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) { fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
readEvents(entity, result, true); readEvents(entity, result, true);
if (checkEvents && !TimelineStorageUtils.matchEventFilters( if (checkEvents && !TimelineStorageUtils.matchEventFilters(
entity.getEvents(), eventFilters)) { entity.getEvents(), filters.getEventFilters())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -348,12 +346,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch metrics // fetch metrics
boolean checkMetrics = metricFilters != null && metricFilters.size() > 0; boolean checkMetrics =
filters != null && filters.getMetricFilters() != null &&
filters.getMetricFilters().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) { fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
readMetrics(entity, result, ApplicationColumnPrefix.METRIC); readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
if (checkMetrics && !TimelineStorageUtils.matchMetricFilters( if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
entity.getMetrics(), metricFilters)) { entity.getMetrics(), filters.getMetricFilters())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&

View File

@ -18,9 +18,7 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.reader; package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
@ -32,8 +30,9 @@
import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix; import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
@ -49,24 +48,14 @@ class FlowActivityEntityReader extends TimelineEntityReader {
private static final FlowActivityTable FLOW_ACTIVITY_TABLE = private static final FlowActivityTable FLOW_ACTIVITY_TABLE =
new FlowActivityTable(); new FlowActivityTable();
public FlowActivityEntityReader(String userId, String clusterId, public FlowActivityEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
Long limit, Long createdTimeBegin, Long createdTimeEnd, super(ctxt, entityFilters, toRetrieve, true);
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
EnumSet<Field> fieldsToRetrieve) {
super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo, infoFilters,
configFilters, metricFilters, eventFilters, null, null,
fieldsToRetrieve, true);
} }
public FlowActivityEntityReader(String userId, String clusterId, public FlowActivityEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineDataToRetrieve toRetrieve) {
String entityId, EnumSet<Field> fieldsToRetrieve) { super(ctxt, toRetrieve);
super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
null, null, fieldsToRetrieve);
} }
/** /**
@ -79,21 +68,13 @@ protected BaseTable<?> getTable() {
@Override @Override
protected void validateParams() { protected void validateParams() {
Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null"); Preconditions.checkNotNull(getContext().getClusterId(),
"clusterId shouldn't be null");
} }
@Override @Override
protected void augmentParams(Configuration hbaseConf, Connection conn) protected void augmentParams(Configuration hbaseConf, Connection conn)
throws IOException { throws IOException {
if (limit == null || limit < 0) {
limit = TimelineReader.DEFAULT_LIMIT;
}
if (createdTimeBegin == null) {
createdTimeBegin = DEFAULT_BEGIN_TIME;
}
if (createdTimeEnd == null) {
createdTimeEnd = DEFAULT_END_TIME;
}
} }
@Override @Override
@ -112,20 +93,24 @@ protected Result getResult(Configuration hbaseConf, Connection conn,
protected ResultScanner getResults(Configuration hbaseConf, protected ResultScanner getResults(Configuration hbaseConf,
Connection conn, FilterList filterList) throws IOException { Connection conn, FilterList filterList) throws IOException {
Scan scan = new Scan(); Scan scan = new Scan();
if (createdTimeBegin == DEFAULT_BEGIN_TIME && String clusterId = getContext().getClusterId();
createdTimeEnd == DEFAULT_END_TIME) { if (getFilters().getCreatedTimeBegin() == 0L &&
getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) {
// All records have to be chosen.
scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId)); scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));
} else { } else {
scan.setStartRow( scan.setStartRow(
FlowActivityRowKey.getRowKeyPrefix(clusterId, createdTimeEnd)); FlowActivityRowKey.getRowKeyPrefix(clusterId,
getFilters().getCreatedTimeEnd()));
scan.setStopRow( scan.setStopRow(
FlowActivityRowKey.getRowKeyPrefix(clusterId, FlowActivityRowKey.getRowKeyPrefix(clusterId,
(createdTimeBegin <= 0 ? 0: (createdTimeBegin - 1)))); (getFilters().getCreatedTimeBegin() <= 0 ? 0 :
(getFilters().getCreatedTimeBegin() - 1))));
} }
// use the page filter to limit the result to the page size // use the page filter to limit the result to the page size
// the scanner may still return more than the limit; therefore we need to // the scanner may still return more than the limit; therefore we need to
// read the right number as we iterate // read the right number as we iterate
scan.setFilter(new PageFilter(limit)); scan.setFilter(new PageFilter(getFilters().getLimit()));
return table.getResultScanner(hbaseConf, conn, scan); return table.getResultScanner(hbaseConf, conn, scan);
} }
@ -137,8 +122,8 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
String user = rowKey.getUserId(); String user = rowKey.getUserId();
String flowName = rowKey.getFlowName(); String flowName = rowKey.getFlowName();
FlowActivityEntity flowActivity = FlowActivityEntity flowActivity = new FlowActivityEntity(
new FlowActivityEntity(clusterId, time, user, flowName); getContext().getClusterId(), time, user, flowName);
// set the id // set the id
flowActivity.setId(flowActivity.getId()); flowActivity.setId(flowActivity.getId());
// get the list of run ids along with the version that are associated with // get the list of run ids along with the version that are associated with

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.reader; package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
@ -38,9 +35,10 @@
import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn; import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
@ -58,26 +56,14 @@
class FlowRunEntityReader extends TimelineEntityReader { class FlowRunEntityReader extends TimelineEntityReader {
private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable(); private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable();
public FlowRunEntityReader(String userId, String clusterId, public FlowRunEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
Long limit, Long createdTimeBegin, Long createdTimeEnd, super(ctxt, entityFilters, toRetrieve, true);
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve) {
super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo, infoFilters,
configFilters, metricFilters, eventFilters, null, metricsToRetrieve,
fieldsToRetrieve, true);
} }
public FlowRunEntityReader(String userId, String clusterId, public FlowRunEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineDataToRetrieve toRetrieve) {
String entityId, TimelineFilterList confsToRetrieve, super(ctxt, toRetrieve);
TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
null, metricsToRetrieve, fieldsToRetrieve);
} }
/** /**
@ -90,35 +76,21 @@ protected BaseTable<?> getTable() {
@Override @Override
protected void validateParams() { protected void validateParams() {
Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null"); Preconditions.checkNotNull(getContext().getClusterId(),
Preconditions.checkNotNull(userId, "userId shouldn't be null"); "clusterId shouldn't be null");
Preconditions.checkNotNull(flowName, "flowName shouldn't be null"); Preconditions.checkNotNull(getContext().getUserId(),
"userId shouldn't be null");
Preconditions.checkNotNull(getContext().getFlowName(),
"flowName shouldn't be null");
if (singleEntityRead) { if (singleEntityRead) {
Preconditions.checkNotNull(flowRunId, "flowRunId shouldn't be null"); Preconditions.checkNotNull(getContext().getFlowRunId(),
"flowRunId shouldn't be null");
} }
} }
@Override @Override
protected void augmentParams(Configuration hbaseConf, Connection conn) { protected void augmentParams(Configuration hbaseConf, Connection conn) {
if (!singleEntityRead) { getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve();
if (fieldsToRetrieve == null) {
fieldsToRetrieve = EnumSet.noneOf(Field.class);
}
if (limit == null || limit < 0) {
limit = TimelineReader.DEFAULT_LIMIT;
}
if (createdTimeBegin == null) {
createdTimeBegin = DEFAULT_BEGIN_TIME;
}
if (createdTimeEnd == null) {
createdTimeEnd = DEFAULT_END_TIME;
}
if (!fieldsToRetrieve.contains(Field.METRICS) &&
metricsToRetrieve != null &&
!metricsToRetrieve.getFilterList().isEmpty()) {
fieldsToRetrieve.add(Field.METRICS);
}
}
} }
@Override @Override
@ -129,9 +101,11 @@ protected FilterList constructFilterListBasedOnFields() {
FamilyFilter infoColumnFamily = FamilyFilter infoColumnFamily =
new FamilyFilter(CompareOp.EQUAL, new FamilyFilter(CompareOp.EQUAL,
new BinaryComparator(FlowRunColumnFamily.INFO.getBytes())); new BinaryComparator(FlowRunColumnFamily.INFO.getBytes()));
TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
// Metrics not required. // Metrics not required.
if (!singleEntityRead && !fieldsToRetrieve.contains(Field.METRICS) && if (!singleEntityRead &&
!fieldsToRetrieve.contains(Field.ALL)) { !dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) &&
!dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL)) {
FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE); FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE);
infoColFamilyList.addFilter(infoColumnFamily); infoColFamilyList.addFilter(infoColumnFamily);
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
@ -140,12 +114,12 @@ protected FilterList constructFilterListBasedOnFields() {
FlowRunColumnPrefix.METRIC.getColumnPrefixBytes("")))); FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(""))));
list.addFilter(infoColFamilyList); list.addFilter(infoColFamilyList);
} }
if (metricsToRetrieve != null && if (dataToRetrieve.getMetricsToRetrieve() != null &&
!metricsToRetrieve.getFilterList().isEmpty()) { !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) {
FilterList infoColFamilyList = new FilterList(); FilterList infoColFamilyList = new FilterList();
infoColFamilyList.addFilter(infoColumnFamily); infoColFamilyList.addFilter(infoColumnFamily);
infoColFamilyList.addFilter(TimelineFilterUtils.createHBaseFilterList( infoColFamilyList.addFilter(TimelineFilterUtils.createHBaseFilterList(
FlowRunColumnPrefix.METRIC, metricsToRetrieve)); FlowRunColumnPrefix.METRIC, dataToRetrieve.getMetricsToRetrieve()));
list.addFilter(infoColFamilyList); list.addFilter(infoColFamilyList);
} }
return list; return list;
@ -154,8 +128,10 @@ protected FilterList constructFilterListBasedOnFields() {
@Override @Override
protected Result getResult(Configuration hbaseConf, Connection conn, protected Result getResult(Configuration hbaseConf, Connection conn,
FilterList filterList) throws IOException { FilterList filterList) throws IOException {
TimelineReaderContext context = getContext();
byte[] rowKey = byte[] rowKey =
FlowRunRowKey.getRowKey(clusterId, userId, flowName, flowRunId); FlowRunRowKey.getRowKey(context.getClusterId(), context.getUserId(),
context.getFlowName(), context.getFlowRunId());
Get get = new Get(rowKey); Get get = new Get(rowKey);
get.setMaxVersions(Integer.MAX_VALUE); get.setMaxVersions(Integer.MAX_VALUE);
if (filterList != null && !filterList.getFilters().isEmpty()) { if (filterList != null && !filterList.getFilters().isEmpty()) {
@ -168,10 +144,12 @@ protected Result getResult(Configuration hbaseConf, Connection conn,
protected ResultScanner getResults(Configuration hbaseConf, protected ResultScanner getResults(Configuration hbaseConf,
Connection conn, FilterList filterList) throws IOException { Connection conn, FilterList filterList) throws IOException {
Scan scan = new Scan(); Scan scan = new Scan();
TimelineReaderContext context = getContext();
scan.setRowPrefixFilter( scan.setRowPrefixFilter(
FlowRunRowKey.getRowKeyPrefix(clusterId, userId, flowName)); FlowRunRowKey.getRowKeyPrefix(context.getClusterId(),
context.getUserId(), context.getFlowName()));
FilterList newList = new FilterList(); FilterList newList = new FilterList();
newList.addFilter(new PageFilter(limit)); newList.addFilter(new PageFilter(getFilters().getLimit()));
if (filterList != null && !filterList.getFilters().isEmpty()) { if (filterList != null && !filterList.getFilters().isEmpty()) {
newList.addFilter(filterList); newList.addFilter(filterList);
} }
@ -181,11 +159,12 @@ protected ResultScanner getResults(Configuration hbaseConf,
@Override @Override
protected TimelineEntity parseEntity(Result result) throws IOException { protected TimelineEntity parseEntity(Result result) throws IOException {
TimelineReaderContext context = getContext();
FlowRunEntity flowRun = new FlowRunEntity(); FlowRunEntity flowRun = new FlowRunEntity();
flowRun.setUser(userId); flowRun.setUser(context.getUserId());
flowRun.setName(flowName); flowRun.setName(context.getFlowName());
if (singleEntityRead) { if (singleEntityRead) {
flowRun.setRunId(flowRunId); flowRun.setRunId(context.getFlowRunId());
} else { } else {
FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow()); FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow());
flowRun.setRunId(rowKey.getFlowRunId()); flowRun.setRunId(rowKey.getFlowRunId());
@ -196,8 +175,9 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
if (startTime != null) { if (startTime != null) {
flowRun.setStartTime(startTime.longValue()); flowRun.setStartTime(startTime.longValue());
} }
if (!singleEntityRead && (flowRun.getStartTime() < createdTimeBegin || if (!singleEntityRead &&
flowRun.getStartTime() > createdTimeEnd)) { (flowRun.getStartTime() < getFilters().getCreatedTimeBegin() ||
flowRun.getStartTime() > getFilters().getCreatedTimeEnd())) {
return null; return null;
} }
@ -214,7 +194,8 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// read metrics // read metrics
if (singleEntityRead || fieldsToRetrieve.contains(Field.METRICS)) { if (singleEntityRead ||
getDataToRetrieve().getFieldsToRetrieve().contains(Field.METRICS)) {
readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC); readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
} }

View File

@ -42,9 +42,10 @@
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
@ -77,26 +78,15 @@ class GenericEntityReader extends TimelineEntityReader {
*/ */
private final AppToFlowTable appToFlowTable = new AppToFlowTable(); private final AppToFlowTable appToFlowTable = new AppToFlowTable();
public GenericEntityReader(String userId, String clusterId, public GenericEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve,
Long limit, Long createdTimeBegin, Long createdTimeEnd, boolean sortedKeys) {
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo, super(ctxt, entityFilters, toRetrieve, sortedKeys);
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo, infoFilters,
configFilters, metricFilters, eventFilters, confsToRetrieve,
metricsToRetrieve, fieldsToRetrieve, sortedKeys);
} }
public GenericEntityReader(String userId, String clusterId, public GenericEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineDataToRetrieve toRetrieve) {
String entityId, TimelineFilterList confsToRetrieve, super(ctxt, toRetrieve);
TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
} }
/** /**
@ -109,12 +99,13 @@ protected BaseTable<?> getTable() {
@Override @Override
protected FilterList constructFilterListBasedOnFields() { protected FilterList constructFilterListBasedOnFields() {
FilterList list = new FilterList(Operator.MUST_PASS_ONE); FilterList list = new FilterList(Operator.MUST_PASS_ONE);
TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
// Fetch all the columns. // Fetch all the columns.
if (fieldsToRetrieve.contains(Field.ALL) && if (dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(confsToRetrieve == null || (dataToRetrieve.getConfsToRetrieve() == null ||
confsToRetrieve.getFilterList().isEmpty()) && dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) &&
(metricsToRetrieve == null || (dataToRetrieve.getMetricsToRetrieve() == null ||
metricsToRetrieve.getFilterList().isEmpty())) { dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) {
return list; return list;
} }
FilterList infoColFamilyList = new FilterList(); FilterList infoColFamilyList = new FilterList();
@ -123,62 +114,69 @@ protected FilterList constructFilterListBasedOnFields() {
new FamilyFilter(CompareOp.EQUAL, new FamilyFilter(CompareOp.EQUAL,
new BinaryComparator(EntityColumnFamily.INFO.getBytes())); new BinaryComparator(EntityColumnFamily.INFO.getBytes()));
infoColFamilyList.addFilter(infoColumnFamily); infoColFamilyList.addFilter(infoColumnFamily);
TimelineEntityFilters filters = getFilters();
// Events not required. // Events not required.
if (!fieldsToRetrieve.contains(Field.EVENTS) && if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.EVENTS) &&
!fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) { !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(singleEntityRead || filters.getEventFilters() == null)) {
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL, new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator( new BinaryPrefixComparator(
EntityColumnPrefix.EVENT.getColumnPrefixBytes("")))); EntityColumnPrefix.EVENT.getColumnPrefixBytes(""))));
} }
// info not required. // info not required.
if (!fieldsToRetrieve.contains(Field.INFO) && if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.INFO) &&
!fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) { !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(singleEntityRead || filters.getInfoFilters() == null)) {
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL, new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator( new BinaryPrefixComparator(
EntityColumnPrefix.INFO.getColumnPrefixBytes("")))); EntityColumnPrefix.INFO.getColumnPrefixBytes(""))));
} }
// is related to not required. // is related to not required.
if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) && if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.IS_RELATED_TO) &&
!fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) { !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(singleEntityRead || filters.getIsRelatedTo() == null)) {
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL, new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator( new BinaryPrefixComparator(
EntityColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes("")))); EntityColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
} }
// relates to not required. // relates to not required.
if (!fieldsToRetrieve.contains(Field.RELATES_TO) && if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.RELATES_TO) &&
!fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) { !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
(singleEntityRead || filters.getRelatesTo() == null)) {
infoColFamilyList.addFilter( infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL, new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator( new BinaryPrefixComparator(
EntityColumnPrefix.RELATES_TO.getColumnPrefixBytes("")))); EntityColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
} }
list.addFilter(infoColFamilyList); list.addFilter(infoColFamilyList);
if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) || if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS) ||
(confsToRetrieve != null && (!singleEntityRead && filters.getConfigFilters() != null)) ||
!confsToRetrieve.getFilterList().isEmpty())) { (dataToRetrieve.getConfsToRetrieve() != null &&
!dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty())) {
FilterList filterCfg = FilterList filterCfg =
new FilterList(new FamilyFilter(CompareOp.EQUAL, new FilterList(new FamilyFilter(CompareOp.EQUAL,
new BinaryComparator(EntityColumnFamily.CONFIGS.getBytes()))); new BinaryComparator(EntityColumnFamily.CONFIGS.getBytes())));
if (confsToRetrieve != null && if (dataToRetrieve.getConfsToRetrieve() != null &&
!confsToRetrieve.getFilterList().isEmpty()) { !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) {
filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList( filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
EntityColumnPrefix.CONFIG, confsToRetrieve)); EntityColumnPrefix.CONFIG, dataToRetrieve.getConfsToRetrieve()));
} }
list.addFilter(filterCfg); list.addFilter(filterCfg);
} }
if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) || if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) ||
(metricsToRetrieve != null && (!singleEntityRead && filters.getMetricFilters() != null)) ||
!metricsToRetrieve.getFilterList().isEmpty())) { (dataToRetrieve.getMetricsToRetrieve() != null &&
!dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) {
FilterList filterMetrics = FilterList filterMetrics =
new FilterList(new FamilyFilter(CompareOp.EQUAL, new FilterList(new FamilyFilter(CompareOp.EQUAL,
new BinaryComparator(EntityColumnFamily.METRICS.getBytes()))); new BinaryComparator(EntityColumnFamily.METRICS.getBytes())));
if (metricsToRetrieve != null && if (dataToRetrieve.getMetricsToRetrieve() != null &&
!metricsToRetrieve.getFilterList().isEmpty()) { !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) {
filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList( filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
EntityColumnPrefix.METRIC, metricsToRetrieve)); EntityColumnPrefix.METRIC, dataToRetrieve.getMetricsToRetrieve()));
} }
list.addFilter(filterMetrics); list.addFilter(filterMetrics);
} }
@ -215,56 +213,42 @@ public FlowContext(String user, String flowName, Long flowRunId) {
@Override @Override
protected void validateParams() { protected void validateParams() {
Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null"); Preconditions.checkNotNull(getContext().getClusterId(),
Preconditions.checkNotNull(appId, "appId shouldn't be null"); "clusterId shouldn't be null");
Preconditions.checkNotNull(entityType, "entityType shouldn't be null"); Preconditions.checkNotNull(getContext().getAppId(),
"appId shouldn't be null");
Preconditions.checkNotNull(getContext().getEntityType(),
"entityType shouldn't be null");
if (singleEntityRead) { if (singleEntityRead) {
Preconditions.checkNotNull(entityId, "entityId shouldn't be null"); Preconditions.checkNotNull(getContext().getEntityId(),
"entityId shouldn't be null");
} }
} }
@Override @Override
protected void augmentParams(Configuration hbaseConf, Connection conn) protected void augmentParams(Configuration hbaseConf, Connection conn)
throws IOException { throws IOException {
TimelineReaderContext context = getContext();
// In reality all three should be null or neither should be null // In reality all three should be null or neither should be null
if (flowName == null || flowRunId == null || userId == null) { if (context.getFlowName() == null || context.getFlowRunId() == null ||
FlowContext context = context.getUserId() == null) {
lookupFlowContext(clusterId, appId, hbaseConf, conn); FlowContext flowContext = lookupFlowContext(
flowName = context.flowName; context.getClusterId(), context.getAppId(), hbaseConf, conn);
flowRunId = context.flowRunId; context.setFlowName(flowContext.flowName);
userId = context.userId; context.setFlowRunId(flowContext.flowRunId);
} context.setUserId(flowContext.userId);
if (fieldsToRetrieve == null) {
fieldsToRetrieve = EnumSet.noneOf(Field.class);
}
if (!fieldsToRetrieve.contains(Field.CONFIGS) &&
confsToRetrieve != null && !confsToRetrieve.getFilterList().isEmpty()) {
fieldsToRetrieve.add(Field.CONFIGS);
}
if (!fieldsToRetrieve.contains(Field.METRICS) &&
metricsToRetrieve != null &&
!metricsToRetrieve.getFilterList().isEmpty()) {
fieldsToRetrieve.add(Field.METRICS);
}
if (!singleEntityRead) {
if (limit == null || limit < 0) {
limit = TimelineReader.DEFAULT_LIMIT;
}
if (createdTimeBegin == null) {
createdTimeBegin = DEFAULT_BEGIN_TIME;
}
if (createdTimeEnd == null) {
createdTimeEnd = DEFAULT_END_TIME;
}
} }
getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve();
} }
@Override @Override
protected Result getResult(Configuration hbaseConf, Connection conn, protected Result getResult(Configuration hbaseConf, Connection conn,
FilterList filterList) throws IOException { FilterList filterList) throws IOException {
TimelineReaderContext context = getContext();
byte[] rowKey = byte[] rowKey =
EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId, EntityRowKey.getRowKey(context.getClusterId(), context.getUserId(),
entityType, entityId); context.getFlowName(), context.getFlowRunId(), context.getAppId(),
context.getEntityType(), context.getEntityId());
Get get = new Get(rowKey); Get get = new Get(rowKey);
get.setMaxVersions(Integer.MAX_VALUE); get.setMaxVersions(Integer.MAX_VALUE);
if (filterList != null && !filterList.getFilters().isEmpty()) { if (filterList != null && !filterList.getFilters().isEmpty()) {
@ -279,8 +263,10 @@ protected ResultScanner getResults(Configuration hbaseConf,
// Scan through part of the table to find the entities belong to one app // Scan through part of the table to find the entities belong to one app
// and one type // and one type
Scan scan = new Scan(); Scan scan = new Scan();
TimelineReaderContext context = getContext();
scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix( scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
clusterId, userId, flowName, flowRunId, appId, entityType)); context.getClusterId(), context.getUserId(), context.getFlowName(),
context.getFlowRunId(), context.getAppId(), context.getEntityType()));
scan.setMaxVersions(Integer.MAX_VALUE); scan.setMaxVersions(Integer.MAX_VALUE);
if (filterList != null && !filterList.getFilters().isEmpty()) { if (filterList != null && !filterList.getFilters().isEmpty()) {
scan.setFilter(filterList); scan.setFilter(filterList);
@ -299,21 +285,25 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
String entityId = EntityColumn.ID.readResult(result).toString(); String entityId = EntityColumn.ID.readResult(result).toString();
entity.setId(entityId); entity.setId(entityId);
TimelineEntityFilters filters = getFilters();
// fetch created time // fetch created time
Number createdTime = (Number)EntityColumn.CREATED_TIME.readResult(result); Number createdTime = (Number)EntityColumn.CREATED_TIME.readResult(result);
entity.setCreatedTime(createdTime.longValue()); entity.setCreatedTime(createdTime.longValue());
if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin || if (!singleEntityRead &&
entity.getCreatedTime() > createdTimeEnd)) { (entity.getCreatedTime() < filters.getCreatedTimeBegin() ||
entity.getCreatedTime() > filters.getCreatedTimeEnd())) {
return null; return null;
} }
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
// fetch is related to entities // fetch is related to entities
boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0; boolean checkIsRelatedTo =
filters != null && filters.getIsRelatedTo() != null &&
filters.getIsRelatedTo().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) { fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true); readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations( if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations(
entity.getIsRelatedToEntities(), isRelatedTo)) { entity.getIsRelatedToEntities(), filters.getIsRelatedTo())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -323,12 +313,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch relates to entities // fetch relates to entities
boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0; boolean checkRelatesTo =
filters != null && filters.getRelatesTo() != null &&
filters.getRelatesTo().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) { fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false); readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
if (checkRelatesTo && !TimelineStorageUtils.matchRelations( if (checkRelatesTo && !TimelineStorageUtils.matchRelations(
entity.getRelatesToEntities(), relatesTo)) { entity.getRelatesToEntities(), filters.getRelatesTo())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -338,12 +330,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch info // fetch info
boolean checkInfo = infoFilters != null && infoFilters.size() > 0; boolean checkInfo = filters != null && filters.getInfoFilters() != null &&
filters.getInfoFilters().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.INFO) || checkInfo) { fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false); readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
if (checkInfo && if (checkInfo &&
!TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) { !TimelineStorageUtils.matchFilters(
entity.getInfo(), filters.getInfoFilters())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -353,12 +347,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch configs // fetch configs
boolean checkConfigs = configFilters != null && configFilters.size() > 0; boolean checkConfigs =
filters != null && filters.getConfigFilters() != null &&
filters.getConfigFilters().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) { fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true); readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
if (checkConfigs && !TimelineStorageUtils.matchFilters( if (checkConfigs && !TimelineStorageUtils.matchFilters(
entity.getConfigs(), configFilters)) { entity.getConfigs(), filters.getConfigFilters())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -368,12 +364,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch events // fetch events
boolean checkEvents = eventFilters != null && eventFilters.size() > 0; boolean checkEvents =
filters != null && filters.getEventFilters() != null &&
filters.getEventFilters().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) { fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
readEvents(entity, result, false); readEvents(entity, result, false);
if (checkEvents && !TimelineStorageUtils.matchEventFilters( if (checkEvents && !TimelineStorageUtils.matchEventFilters(
entity.getEvents(), eventFilters)) { entity.getEvents(), filters.getEventFilters())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&
@ -383,12 +381,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
} }
// fetch metrics // fetch metrics
boolean checkMetrics = metricFilters != null && metricFilters.size() > 0; boolean checkMetrics =
filters != null && filters.getMetricFilters() != null &&
filters.getMetricFilters().size() > 0;
if (fieldsToRetrieve.contains(Field.ALL) || if (fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) { fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
readMetrics(entity, result, EntityColumnPrefix.METRIC); readMetrics(entity, result, EntityColumnPrefix.METRIC);
if (checkMetrics && !TimelineStorageUtils.matchMetricFilters( if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
entity.getMetrics(), metricFilters)) { entity.getMetrics(), filters.getMetricFilters())) {
return null; return null;
} }
if (!fieldsToRetrieve.contains(Field.ALL) && if (!fieldsToRetrieve.contains(Field.ALL) &&

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.reader; package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet;
import java.util.Map; import java.util.Map;
import java.util.NavigableMap; import java.util.NavigableMap;
import java.util.NavigableSet; import java.util.NavigableSet;
@ -34,8 +33,9 @@
import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
@ -46,32 +46,12 @@
*/ */
public abstract class TimelineEntityReader { public abstract class TimelineEntityReader {
private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class); private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class);
protected static final long DEFAULT_BEGIN_TIME = 0L;
protected static final long DEFAULT_END_TIME = Long.MAX_VALUE;
protected final boolean singleEntityRead; protected final boolean singleEntityRead;
private TimelineReaderContext context;
protected String userId; private TimelineDataToRetrieve dataToRetrieve;
protected String clusterId;
protected String flowName;
protected Long flowRunId;
protected String appId;
protected String entityType;
protected EnumSet<Field> fieldsToRetrieve;
// used only for a single entity read mode
protected String entityId;
// used only for multiple entity read mode // used only for multiple entity read mode
protected Long limit; private TimelineEntityFilters filters;
protected Long createdTimeBegin;
protected Long createdTimeEnd;
protected Map<String, Set<String>> relatesTo;
protected Map<String, Set<String>> isRelatedTo;
protected Map<String, Object> infoFilters;
protected Map<String, String> configFilters;
protected Set<String> metricFilters;
protected Set<String> eventFilters;
protected TimelineFilterList confsToRetrieve;
protected TimelineFilterList metricsToRetrieve;
/** /**
* Main table the entity reader uses. * Main table the entity reader uses.
@ -89,34 +69,14 @@ public abstract class TimelineEntityReader {
/** /**
* Instantiates a reader for multiple-entity reads. * Instantiates a reader for multiple-entity reads.
*/ */
protected TimelineEntityReader(String userId, String clusterId, protected TimelineEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve,
Long limit, Long createdTimeBegin, Long createdTimeEnd, boolean sortedKeys) {
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
this.singleEntityRead = false; this.singleEntityRead = false;
this.sortedKeys = sortedKeys; this.sortedKeys = sortedKeys;
this.userId = userId; this.context = ctxt;
this.clusterId = clusterId; this.dataToRetrieve = toRetrieve;
this.flowName = flowName; this.filters = entityFilters;
this.flowRunId = flowRunId;
this.appId = appId;
this.entityType = entityType;
this.fieldsToRetrieve = fieldsToRetrieve;
this.limit = limit;
this.createdTimeBegin = createdTimeBegin;
this.createdTimeEnd = createdTimeEnd;
this.relatesTo = relatesTo;
this.isRelatedTo = isRelatedTo;
this.infoFilters = infoFilters;
this.configFilters = configFilters;
this.metricFilters = metricFilters;
this.eventFilters = eventFilters;
this.confsToRetrieve = confsToRetrieve;
this.metricsToRetrieve = metricsToRetrieve;
this.table = getTable(); this.table = getTable();
} }
@ -124,21 +84,11 @@ protected TimelineEntityReader(String userId, String clusterId,
/** /**
* Instantiates a reader for single-entity reads. * Instantiates a reader for single-entity reads.
*/ */
protected TimelineEntityReader(String userId, String clusterId, protected TimelineEntityReader(TimelineReaderContext ctxt,
String flowName, Long flowRunId, String appId, String entityType, TimelineDataToRetrieve toRetrieve) {
String entityId, TimelineFilterList confsToRetrieve,
TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
this.singleEntityRead = true; this.singleEntityRead = true;
this.userId = userId; this.context = ctxt;
this.clusterId = clusterId; this.dataToRetrieve = toRetrieve;
this.flowName = flowName;
this.flowRunId = flowRunId;
this.appId = appId;
this.entityType = entityType;
this.fieldsToRetrieve = fieldsToRetrieve;
this.entityId = entityId;
this.confsToRetrieve = confsToRetrieve;
this.metricsToRetrieve = metricsToRetrieve;
this.table = getTable(); this.table = getTable();
} }
@ -151,6 +101,18 @@ protected TimelineEntityReader(String userId, String clusterId,
*/ */
protected abstract FilterList constructFilterListBasedOnFields(); protected abstract FilterList constructFilterListBasedOnFields();
protected TimelineReaderContext getContext() {
return context;
}
protected TimelineDataToRetrieve getDataToRetrieve() {
return dataToRetrieve;
}
protected TimelineEntityFilters getFilters() {
return filters;
}
/** /**
* Reads and deserializes a single timeline entity from the HBase storage. * Reads and deserializes a single timeline entity from the HBase storage.
*/ */
@ -163,7 +125,8 @@ public TimelineEntity readEntity(Configuration hbaseConf, Connection conn)
Result result = getResult(hbaseConf, conn, filterList); Result result = getResult(hbaseConf, conn, filterList);
if (result == null || result.isEmpty()) { if (result == null || result.isEmpty()) {
// Could not find a matching row. // Could not find a matching row.
LOG.info("Cannot find matching entity of type " + entityType); LOG.info("Cannot find matching entity of type " +
context.getEntityType());
return null; return null;
} }
return parseEntity(result); return parseEntity(result);
@ -190,11 +153,11 @@ public Set<TimelineEntity> readEntities(Configuration hbaseConf,
} }
entities.add(entity); entities.add(entity);
if (!sortedKeys) { if (!sortedKeys) {
if (entities.size() > limit) { if (entities.size() > filters.getLimit()) {
entities.pollLast(); entities.pollLast();
} }
} else { } else {
if (entities.size() == limit) { if (entities.size() == filters.getLimit()) {
break; break;
} }
} }

View File

@ -17,13 +17,10 @@
*/ */
package org.apache.hadoop.yarn.server.timelineservice.storage.reader; package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
import java.util.EnumSet;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
/** /**
* Factory methods for instantiating a timeline entity reader. * Factory methods for instantiating a timeline entity reader.
@ -33,25 +30,21 @@ public class TimelineEntityReaderFactory {
* Creates a timeline entity reader instance for reading a single entity with * Creates a timeline entity reader instance for reading a single entity with
* the specified input. * the specified input.
*/ */
public static TimelineEntityReader createSingleEntityReader(String userId, public static TimelineEntityReader createSingleEntityReader(
String clusterId, String flowName, Long flowRunId, String appId, TimelineReaderContext context, TimelineDataToRetrieve dataToRetrieve) {
String entityType, String entityId, TimelineFilterList confs,
TimelineFilterList metrics, EnumSet<Field> fieldsToRetrieve) {
// currently the types that are handled separate from the generic entity // currently the types that are handled separate from the generic entity
// table are application, flow run, and flow activity entities // table are application, flow run, and flow activity entities
if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) { if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) {
return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId, return new ApplicationEntityReader(context, dataToRetrieve);
appId, entityType, entityId, confs, metrics, fieldsToRetrieve); } else if (TimelineEntityType.
} else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) { YARN_FLOW_RUN.matches(context.getEntityType())) {
return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId, return new FlowRunEntityReader(context, dataToRetrieve);
appId, entityType, entityId, confs, metrics, fieldsToRetrieve); } else if (TimelineEntityType.
} else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) { YARN_FLOW_ACTIVITY.matches(context.getEntityType())) {
return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId, return new FlowActivityEntityReader(context, dataToRetrieve);
appId, entityType, entityId, fieldsToRetrieve);
} else { } else {
// assume we're dealing with a generic entity read // assume we're dealing with a generic entity read
return new GenericEntityReader(userId, clusterId, flowName, flowRunId, return new GenericEntityReader(context, dataToRetrieve);
appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
} }
} }
@ -59,37 +52,22 @@ public static TimelineEntityReader createSingleEntityReader(String userId,
* Creates a timeline entity reader instance for reading set of entities with * Creates a timeline entity reader instance for reading set of entities with
* the specified input and predicates. * the specified input and predicates.
*/ */
public static TimelineEntityReader createMultipleEntitiesReader(String userId, public static TimelineEntityReader createMultipleEntitiesReader(
String clusterId, String flowName, Long flowRunId, String appId, TimelineReaderContext context, TimelineEntityFilters filters,
String entityType, Long limit, Long createdTimeBegin, Long createdTimeEnd, TimelineDataToRetrieve dataToRetrieve) {
Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
Map<String, Object> infoFilters, Map<String, String> configFilters,
Set<String> metricFilters, Set<String> eventFilters,
TimelineFilterList confs, TimelineFilterList metrics,
EnumSet<Field> fieldsToRetrieve) {
// currently the types that are handled separate from the generic entity // currently the types that are handled separate from the generic entity
// table are application, flow run, and flow activity entities // table are application, flow run, and flow activity entities
if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) { if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) {
return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId, return new ApplicationEntityReader(context, filters, dataToRetrieve);
appId, entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo, } else if (TimelineEntityType.
isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters, YARN_FLOW_ACTIVITY.matches(context.getEntityType())) {
confs, metrics, fieldsToRetrieve); return new FlowActivityEntityReader(context, filters, dataToRetrieve);
} else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) { } else if (TimelineEntityType.
return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId, YARN_FLOW_RUN.matches(context.getEntityType())) {
appId, entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo, return new FlowRunEntityReader(context, filters, dataToRetrieve);
isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters,
fieldsToRetrieve);
} else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId,
appId, entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo,
isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters,
confs, metrics, fieldsToRetrieve);
} else { } else {
// assume we're dealing with a generic entity read // assume we're dealing with a generic entity read
return new GenericEntityReader(userId, clusterId, flowName, flowRunId, return new GenericEntityReader(context, filters, dataToRetrieve, false);
appId, entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo,
isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters,
confs, metrics, fieldsToRetrieve, false);
} }
} }
} }

View File

@ -37,6 +37,9 @@
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -258,9 +261,10 @@ public TimelineReader getTimelineReader() {
public void testGetEntityDefaultView() throws Exception { public void testGetEntityDefaultView() throws Exception {
// If no fields are specified, entity is returned with default view i.e. // If no fields are specified, entity is returned with default view i.e.
// only the id, type and created time. // only the id, type and created time.
TimelineEntity result = TimelineEntity result = reader.getEntity(
reader.getEntity("user1", "cluster1", "flow1", 1L, "app1", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
"app", "id_1", null, null, null); "app", "id_1"),
new TimelineDataToRetrieve(null, null, null));
Assert.assertEquals( Assert.assertEquals(
(new TimelineEntity.Identifier("app", "id_1")).toString(), (new TimelineEntity.Identifier("app", "id_1")).toString(),
result.getIdentifier().toString()); result.getIdentifier().toString());
@ -272,9 +276,10 @@ public void testGetEntityDefaultView() throws Exception {
@Test @Test
public void testGetEntityByClusterAndApp() throws Exception { public void testGetEntityByClusterAndApp() throws Exception {
// Cluster and AppId should be enough to get an entity. // Cluster and AppId should be enough to get an entity.
TimelineEntity result = TimelineEntity result = reader.getEntity(
reader.getEntity(null, "cluster1", null, null, "app1", new TimelineReaderContext("cluster1", null, null, null, "app1", "app",
"app", "id_1", null, null, null); "id_1"),
new TimelineDataToRetrieve(null, null, null));
Assert.assertEquals( Assert.assertEquals(
(new TimelineEntity.Identifier("app", "id_1")).toString(), (new TimelineEntity.Identifier("app", "id_1")).toString(),
result.getIdentifier().toString()); result.getIdentifier().toString());
@ -288,9 +293,10 @@ public void testGetEntityByClusterAndApp() throws Exception {
public void testAppFlowMappingCsv() throws Exception { public void testAppFlowMappingCsv() throws Exception {
// Test getting an entity by cluster and app where flow entry // Test getting an entity by cluster and app where flow entry
// in app flow mapping csv has commas. // in app flow mapping csv has commas.
TimelineEntity result = TimelineEntity result = reader.getEntity(
reader.getEntity(null, "cluster1", null, null, "app2", new TimelineReaderContext("cluster1", null, null, null, "app2",
"app", "id_5", null, null, null); "app", "id_5"),
new TimelineDataToRetrieve(null, null, null));
Assert.assertEquals( Assert.assertEquals(
(new TimelineEntity.Identifier("app", "id_5")).toString(), (new TimelineEntity.Identifier("app", "id_5")).toString(),
result.getIdentifier().toString()); result.getIdentifier().toString());
@ -300,10 +306,11 @@ public void testAppFlowMappingCsv() throws Exception {
@Test @Test
public void testGetEntityCustomFields() throws Exception { public void testGetEntityCustomFields() throws Exception {
// Specified fields in addition to default view will be returned. // Specified fields in addition to default view will be returned.
TimelineEntity result = TimelineEntity result = reader.getEntity(
reader.getEntity("user1", "cluster1", "flow1", 1L, new TimelineReaderContext("cluster1","user1", "flow1", 1L, "app1",
"app1", "app", "id_1", null, null, "app", "id_1"),
EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS)); new TimelineDataToRetrieve(null, null,
EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS)));
Assert.assertEquals( Assert.assertEquals(
(new TimelineEntity.Identifier("app", "id_1")).toString(), (new TimelineEntity.Identifier("app", "id_1")).toString(),
result.getIdentifier().toString()); result.getIdentifier().toString());
@ -318,9 +325,10 @@ public void testGetEntityCustomFields() throws Exception {
@Test @Test
public void testGetEntityAllFields() throws Exception { public void testGetEntityAllFields() throws Exception {
// All fields of TimelineEntity will be returned. // All fields of TimelineEntity will be returned.
TimelineEntity result = TimelineEntity result = reader.getEntity(
reader.getEntity("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1","user1", "flow1", 1L, "app1",
"id_1", null, null, EnumSet.of(Field.ALL)); "app", "id_1"),
new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
Assert.assertEquals( Assert.assertEquals(
(new TimelineEntity.Identifier("app", "id_1")).toString(), (new TimelineEntity.Identifier("app", "id_1")).toString(),
result.getIdentifier().toString()); result.getIdentifier().toString());
@ -333,20 +341,21 @@ public void testGetEntityAllFields() throws Exception {
@Test @Test
public void testGetAllEntities() throws Exception { public void testGetAllEntities() throws Exception {
Set<TimelineEntity> result = Set<TimelineEntity> result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, null, null, null, null, null, null, null, null, null, null, "app", null), new TimelineEntityFilters(),
null); new TimelineDataToRetrieve());
// All 3 entities will be returned // All 3 entities will be returned
Assert.assertEquals(4, result.size()); Assert.assertEquals(4, result.size());
} }
@Test @Test
public void testGetEntitiesWithLimit() throws Exception { public void testGetEntitiesWithLimit() throws Exception {
Set<TimelineEntity> result = Set<TimelineEntity> result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
2L, null, null, null, null, null, null, null, null, null, null, "app", null),
null); new TimelineEntityFilters(2L, null, null, null, null, null, null,
null, null), new TimelineDataToRetrieve());
Assert.assertEquals(2, result.size()); Assert.assertEquals(2, result.size());
// Needs to be rewritten once hashcode and equals for // Needs to be rewritten once hashcode and equals for
// TimelineEntity is implemented // TimelineEntity is implemented
@ -357,10 +366,11 @@ public void testGetEntitiesWithLimit() throws Exception {
Assert.fail("Entity not sorted by created time"); Assert.fail("Entity not sorted by created time");
} }
} }
result = result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
3L, null, null, null, null, null, null, null, null, null, null, "app", null),
null); new TimelineEntityFilters(3L, null, null, null, null, null, null,
null, null), new TimelineDataToRetrieve());
// Even though 2 entities out of 4 have same created time, one entity // Even though 2 entities out of 4 have same created time, one entity
// is left out due to limit // is left out due to limit
Assert.assertEquals(3, result.size()); Assert.assertEquals(3, result.size());
@ -369,10 +379,12 @@ public void testGetEntitiesWithLimit() throws Exception {
@Test @Test
public void testGetEntitiesByTimeWindows() throws Exception { public void testGetEntitiesByTimeWindows() throws Exception {
// Get entities based on created time start and end time range. // Get entities based on created time start and end time range.
Set<TimelineEntity> result = Set<TimelineEntity> result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, 1425016502030L, 1425016502060L, null, null, null, null, null, "app", null),
null, null, null, null); new TimelineEntityFilters(null, 1425016502030L, 1425016502060L, null,
null, null, null, null, null),
new TimelineDataToRetrieve());
Assert.assertEquals(1, result.size()); Assert.assertEquals(1, result.size());
// Only one entity with ID id_4 should be returned. // Only one entity with ID id_4 should be returned.
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {
@ -382,10 +394,12 @@ public void testGetEntitiesByTimeWindows() throws Exception {
} }
// Get entities if only created time end is specified. // Get entities if only created time end is specified.
result = result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, null, 1425016502010L, null, null, null, null, null, null, "app", null),
null, null, null); new TimelineEntityFilters(null, null, 1425016502010L, null, null,
null, null, null, null),
new TimelineDataToRetrieve());
Assert.assertEquals(3, result.size()); Assert.assertEquals(3, result.size());
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {
if (entity.getId().equals("id_4")) { if (entity.getId().equals("id_4")) {
@ -394,10 +408,12 @@ public void testGetEntitiesByTimeWindows() throws Exception {
} }
// Get entities if only created time start is specified. // Get entities if only created time start is specified.
result = result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, 1425016502010L, null, null, null, null, null, null, null, "app", null),
null, null, null); new TimelineEntityFilters(null, 1425016502010L, null, null, null,
null, null, null, null),
new TimelineDataToRetrieve());
Assert.assertEquals(1, result.size()); Assert.assertEquals(1, result.size());
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {
if (!entity.getId().equals("id_4")) { if (!entity.getId().equals("id_4")) {
@ -411,10 +427,12 @@ public void testGetFilteredEntities() throws Exception {
// Get entities based on info filters. // Get entities based on info filters.
Map<String, Object> infoFilters = new HashMap<String, Object>(); Map<String, Object> infoFilters = new HashMap<String, Object>();
infoFilters.put("info2", 3.5); infoFilters.put("info2", 3.5);
Set<TimelineEntity> result = Set<TimelineEntity> result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, null, null, null, null, infoFilters, null, null, null, null, "app", null),
null, null); new TimelineEntityFilters(null, null, null, null, null, infoFilters,
null, null, null),
new TimelineDataToRetrieve());
Assert.assertEquals(1, result.size()); Assert.assertEquals(1, result.size());
// Only one entity with ID id_3 should be returned. // Only one entity with ID id_3 should be returned.
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {
@ -427,10 +445,12 @@ public void testGetFilteredEntities() throws Exception {
Map<String, String> configFilters = new HashMap<String, String>(); Map<String, String> configFilters = new HashMap<String, String>();
configFilters.put("config_1", "123"); configFilters.put("config_1", "123");
configFilters.put("config_3", "abc"); configFilters.put("config_3", "abc");
result = result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, null, null, null, null, null, configFilters, null, null, null, "app", null),
null, null); new TimelineEntityFilters(null, null, null, null, null, null,
configFilters, null, null),
new TimelineDataToRetrieve());
Assert.assertEquals(2, result.size()); Assert.assertEquals(2, result.size());
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {
if (!entity.getId().equals("id_1") && !entity.getId().equals("id_3")) { if (!entity.getId().equals("id_1") && !entity.getId().equals("id_3")) {
@ -442,10 +462,12 @@ public void testGetFilteredEntities() throws Exception {
Set<String> eventFilters = new HashSet<String>(); Set<String> eventFilters = new HashSet<String>();
eventFilters.add("event_2"); eventFilters.add("event_2");
eventFilters.add("event_4"); eventFilters.add("event_4");
result = result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, null, null, null, null, null, null, null, eventFilters, null, "app", null),
null, null); new TimelineEntityFilters(null, null, null, null, null, null, null,
null, eventFilters),
new TimelineDataToRetrieve());
Assert.assertEquals(1, result.size()); Assert.assertEquals(1, result.size());
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {
if (!entity.getId().equals("id_3")) { if (!entity.getId().equals("id_3")) {
@ -456,10 +478,12 @@ public void testGetFilteredEntities() throws Exception {
// Get entities based on metric filters. // Get entities based on metric filters.
Set<String> metricFilters = new HashSet<String>(); Set<String> metricFilters = new HashSet<String>();
metricFilters.add("metric3"); metricFilters.add("metric3");
result = result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, null, null, null, null, null, null, metricFilters, null, null, "app", null),
null, null); new TimelineEntityFilters(null, null, null, null, null, null, null,
metricFilters, null),
new TimelineDataToRetrieve());
Assert.assertEquals(2, result.size()); Assert.assertEquals(2, result.size());
// Two entities with IDs' id_1 and id_2 should be returned. // Two entities with IDs' id_1 and id_2 should be returned.
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {
@ -476,10 +500,12 @@ public void testGetEntitiesByRelations() throws Exception {
Set<String> relatesToIds = new HashSet<String>(); Set<String> relatesToIds = new HashSet<String>();
relatesToIds.add("flow1"); relatesToIds.add("flow1");
relatesTo.put("flow", relatesToIds); relatesTo.put("flow", relatesToIds);
Set<TimelineEntity> result = Set<TimelineEntity> result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, null, null, relatesTo, null, null, null, null, null, null, "app", null),
null, null); new TimelineEntityFilters(null, null, null, relatesTo, null, null,
null, null, null),
new TimelineDataToRetrieve());
Assert.assertEquals(1, result.size()); Assert.assertEquals(1, result.size());
// Only one entity with ID id_1 should be returned. // Only one entity with ID id_1 should be returned.
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {
@ -493,10 +519,12 @@ public void testGetEntitiesByRelations() throws Exception {
Set<String> isRelatedToIds = new HashSet<String>(); Set<String> isRelatedToIds = new HashSet<String>();
isRelatedToIds.add("tid1_2"); isRelatedToIds.add("tid1_2");
isRelatedTo.put("type1", isRelatedToIds); isRelatedTo.put("type1", isRelatedToIds);
result = result = reader.getEntities(
reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
null, null, null, null, isRelatedTo, null, null, null, null, null, "app", null),
null, null); new TimelineEntityFilters(null, null, null, null, isRelatedTo, null,
null, null, null),
new TimelineDataToRetrieve());
Assert.assertEquals(2, result.size()); Assert.assertEquals(2, result.size());
// Two entities with IDs' id_1 and id_3 should be returned. // Two entities with IDs' id_1 and id_3 should be returned.
for (TimelineEntity entity : result) { for (TimelineEntity entity : result) {

View File

@ -49,6 +49,9 @@
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
@ -574,9 +577,11 @@ public void testWriteApplicationToHBase() throws Exception {
matchMetrics(metricValues, metricMap); matchMetrics(metricValues, metricMap);
// read the timeline entity using the reader this time // read the timeline entity using the reader this time
TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appId, TimelineEntity e1 = reader.getEntity(
entity.getType(), entity.getId(), null, null, new TimelineReaderContext(cluster, user, flow, runid, appId,
EnumSet.of(TimelineReader.Field.ALL)); entity.getType(), entity.getId()),
new TimelineDataToRetrieve(
null, null, EnumSet.of(TimelineReader.Field.ALL)));
assertNotNull(e1); assertNotNull(e1);
// verify attributes // verify attributes
@ -771,12 +776,15 @@ public void testWriteEntityToHBase() throws Exception {
assertEquals(16, colCount); assertEquals(16, colCount);
// read the timeline entity using the reader this time // read the timeline entity using the reader this time
TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName, TimelineEntity e1 = reader.getEntity(
entity.getType(), entity.getId(), null, null, new TimelineReaderContext(cluster, user, flow, runid, appName,
EnumSet.of(TimelineReader.Field.ALL)); entity.getType(), entity.getId()),
Set<TimelineEntity> es1 = reader.getEntities(user, cluster, flow, runid, new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
appName, entity.getType(), null, null, null, null, null, null, null, Set<TimelineEntity> es1 = reader.getEntities(
null, null, null, null, EnumSet.of(TimelineReader.Field.ALL)); new TimelineReaderContext(cluster, user, flow, runid, appName,
entity.getType(), null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
assertNotNull(e1); assertNotNull(e1);
assertEquals(1, es1.size()); assertEquals(1, es1.size());
@ -906,12 +914,14 @@ public void testEvents() throws IOException {
} }
// read the timeline entity using the reader this time // read the timeline entity using the reader this time
TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName, TimelineEntity e1 = reader.getEntity(
entity.getType(), entity.getId(), null, null, new TimelineReaderContext(cluster, user, flow, runid, appName,
EnumSet.of(TimelineReader.Field.ALL)); entity.getType(), entity.getId()),
TimelineEntity e2 = reader.getEntity(user, cluster, null, null, appName, new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
entity.getType(), entity.getId(), null, null, TimelineEntity e2 = reader.getEntity(
EnumSet.of(TimelineReader.Field.ALL)); new TimelineReaderContext(cluster, user, null, null, appName,
entity.getType(), entity.getId()),
new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
assertNotNull(e1); assertNotNull(e1);
assertNotNull(e2); assertNotNull(e2);
assertEquals(e1, e2); assertEquals(e1, e2);
@ -1012,12 +1022,15 @@ public void testEventsWithEmptyInfo() throws IOException {
assertEquals(1, rowCount); assertEquals(1, rowCount);
// read the timeline entity using the reader this time // read the timeline entity using the reader this time
TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName, TimelineEntity e1 = reader.getEntity(
entity.getType(), entity.getId(), null, null, new TimelineReaderContext(cluster, user, flow, runid, appName,
EnumSet.of(TimelineReader.Field.ALL)); entity.getType(), entity.getId()),
Set<TimelineEntity> es1 = reader.getEntities(user, cluster, flow, runid, new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
appName, entity.getType(), null, null, null, null, null, null, null, Set<TimelineEntity> es1 = reader.getEntities(
null, null, null, null, EnumSet.of(TimelineReader.Field.ALL)); new TimelineReaderContext(cluster, user, flow, runid, appName,
entity.getType(), null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
assertNotNull(e1); assertNotNull(e1);
assertEquals(1, es1.size()); assertEquals(1, es1.size());
@ -1100,31 +1113,36 @@ public void testNonIntegralMetricValues() throws IOException {
@Test @Test
public void testReadEntities() throws Exception { public void testReadEntities() throws Exception {
TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name", TimelineEntity e1 = reader.getEntity(
1002345678919L, "application_1231111111_1111","world", "hello", null, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
null, EnumSet.of(Field.ALL)); 1002345678919L, "application_1231111111_1111","world", "hello"),
new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
assertNotNull(e1); assertNotNull(e1);
assertEquals(3, e1.getConfigs().size()); assertEquals(3, e1.getConfigs().size());
assertEquals(1, e1.getIsRelatedToEntities().size()); assertEquals(1, e1.getIsRelatedToEntities().size());
Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1", Set<TimelineEntity> es1 = reader.getEntities(
"some_flow_name", 1002345678919L, "application_1231111111_1111","world", new TimelineReaderContext("cluster1", "user1", "some_flow_name",
null, null, null, null, null, null, null, null, null, null, null, 1002345678919L, "application_1231111111_1111","world",
EnumSet.of(Field.ALL)); null), new TimelineEntityFilters(),
new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
assertEquals(3, es1.size()); assertEquals(3, es1.size());
} }
@Test @Test
public void testReadEntitiesDefaultView() throws Exception { public void testReadEntitiesDefaultView() throws Exception {
TimelineEntity e1 = TimelineEntity e1 = reader.getEntity(
reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
"application_1231111111_1111","world", "hello", null, null, null); 1002345678919L, "application_1231111111_1111","world", "hello"),
new TimelineDataToRetrieve());
assertNotNull(e1); assertNotNull(e1);
assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() && assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() && e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
e1.getRelatesToEntities().isEmpty()); e1.getRelatesToEntities().isEmpty());
Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1", Set<TimelineEntity> es1 = reader.getEntities(
"some_flow_name", 1002345678919L, "application_1231111111_1111","world", new TimelineReaderContext("cluster1", "user1", "some_flow_name",
null, null, null, null, null, null, null, null, null, null, null, null); 1002345678919L, "application_1231111111_1111","world", null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve());
assertEquals(3, es1.size()); assertEquals(3, es1.size());
for (TimelineEntity e : es1) { for (TimelineEntity e : es1) {
assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() && assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
@ -1135,17 +1153,20 @@ public void testReadEntitiesDefaultView() throws Exception {
@Test @Test
public void testReadEntitiesByFields() throws Exception { public void testReadEntitiesByFields() throws Exception {
TimelineEntity e1 = TimelineEntity e1 = reader.getEntity(
reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
"application_1231111111_1111","world", "hello", null, null, 1002345678919L, "application_1231111111_1111","world", "hello"),
EnumSet.of(Field.INFO, Field.CONFIGS)); new TimelineDataToRetrieve(
null, null, EnumSet.of(Field.INFO, Field.CONFIGS)));
assertNotNull(e1); assertNotNull(e1);
assertEquals(3, e1.getConfigs().size()); assertEquals(3, e1.getConfigs().size());
assertEquals(0, e1.getIsRelatedToEntities().size()); assertEquals(0, e1.getIsRelatedToEntities().size());
Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1", Set<TimelineEntity> es1 = reader.getEntities(
"some_flow_name", 1002345678919L, "application_1231111111_1111","world", new TimelineReaderContext("cluster1", "user1", "some_flow_name",
null, null, null, null, null, null, null, null, null, null, null, 1002345678919L, "application_1231111111_1111","world", null),
EnumSet.of(Field.IS_RELATED_TO, Field.METRICS)); new TimelineEntityFilters(),
new TimelineDataToRetrieve(
null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS)));
assertEquals(3, es1.size()); assertEquals(3, es1.size());
int metricsCnt = 0; int metricsCnt = 0;
int isRelatedToCnt = 0; int isRelatedToCnt = 0;
@ -1165,14 +1186,17 @@ public void testReadEntitiesConfigPrefix() throws Exception {
TimelineFilterList list = TimelineFilterList list =
new TimelineFilterList(Operator.OR, new TimelineFilterList(Operator.OR,
new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
TimelineEntity e1 = TimelineEntity e1 = reader.getEntity(
reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
"application_1231111111_1111","world", "hello", list, null, null); 1002345678919L, "application_1231111111_1111","world", "hello"),
new TimelineDataToRetrieve(list, null, null));
assertNotNull(e1); assertNotNull(e1);
assertEquals(1, e1.getConfigs().size()); assertEquals(1, e1.getConfigs().size());
Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1", Set<TimelineEntity> es1 = reader.getEntities(
"some_flow_name", 1002345678919L, "application_1231111111_1111","world", new TimelineReaderContext("cluster1", "user1", "some_flow_name",
null, null, null, null, null, null, null, null, null, list, null, null); 1002345678919L, "application_1231111111_1111","world", null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve(list, null, null));
int cfgCnt = 0; int cfgCnt = 0;
for (TimelineEntity entity : es1) { for (TimelineEntity entity : es1) {
cfgCnt += entity.getConfigs().size(); cfgCnt += entity.getConfigs().size();
@ -1186,10 +1210,12 @@ public void testReadEntitiesConfigFilterPrefix() throws Exception {
TimelineFilterList list = TimelineFilterList list =
new TimelineFilterList(Operator.OR, new TimelineFilterList(Operator.OR,
new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
Set<TimelineEntity> entities = reader.getEntities("user1", "cluster1", Set<TimelineEntity> entities = reader.getEntities(
"some_flow_name", 1002345678919L, "application_1231111111_1111","world", new TimelineReaderContext("cluster1", "user1", "some_flow_name",
null, null, null, null, null, null, confFilters, null, null, list, null, 1002345678919L, "application_1231111111_1111","world", null),
null); new TimelineEntityFilters(null, null, null, null, null, null,
confFilters, null, null),
new TimelineDataToRetrieve(list, null, null));
assertEquals(1, entities.size()); assertEquals(1, entities.size());
int cfgCnt = 0; int cfgCnt = 0;
for (TimelineEntity entity : entities) { for (TimelineEntity entity : entities) {
@ -1203,14 +1229,17 @@ public void testReadEntitiesMetricPrefix() throws Exception {
TimelineFilterList list = TimelineFilterList list =
new TimelineFilterList(Operator.OR, new TimelineFilterList(Operator.OR,
new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
TimelineEntity e1 = TimelineEntity e1 = reader.getEntity(
reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
"application_1231111111_1111","world", "hello", null, list, null); 1002345678919L, "application_1231111111_1111","world", "hello"),
new TimelineDataToRetrieve(null, list, null));
assertNotNull(e1); assertNotNull(e1);
assertEquals(1, e1.getMetrics().size()); assertEquals(1, e1.getMetrics().size());
Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1", Set<TimelineEntity> es1 = reader.getEntities(
"some_flow_name", 1002345678919L, "application_1231111111_1111","world", new TimelineReaderContext("cluster1", "user1", "some_flow_name",
null, null, null, null, null, null, null, null, null, null, list, null); 1002345678919L, "application_1231111111_1111","world", null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve(null, list, null));
int metricCnt = 0; int metricCnt = 0;
for (TimelineEntity entity : es1) { for (TimelineEntity entity : es1) {
metricCnt += entity.getMetrics().size(); metricCnt += entity.getMetrics().size();
@ -1224,10 +1253,12 @@ public void testReadEntitiesMetricFilterPrefix() throws Exception {
TimelineFilterList list = TimelineFilterList list =
new TimelineFilterList(Operator.OR, new TimelineFilterList(Operator.OR,
new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
Set<TimelineEntity> entities = reader.getEntities("user1", "cluster1", Set<TimelineEntity> entities = reader.getEntities(
"some_flow_name", 1002345678919L, "application_1231111111_1111","world", new TimelineReaderContext("cluster1","user1", "some_flow_name",
null, null, null, null, null, null, null, metricFilters, null, null, 1002345678919L, "application_1231111111_1111","world", null),
list, null); new TimelineEntityFilters(null, null, null, null, null, null, null,
metricFilters, null),
new TimelineDataToRetrieve(null, list, null));
assertEquals(1, entities.size()); assertEquals(1, entities.size());
int metricCnt = 0; int metricCnt = 0;
for (TimelineEntity entity : entities) { for (TimelineEntity entity : entities) {
@ -1238,33 +1269,40 @@ public void testReadEntitiesMetricFilterPrefix() throws Exception {
@Test @Test
public void testReadApps() throws Exception { public void testReadApps() throws Exception {
TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name", TimelineEntity e1 = reader.getEntity(
new TimelineReaderContext("cluster1", "user1", "some_flow_name",
1002345678919L, "application_1111111111_2222", 1002345678919L, "application_1111111111_2222",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, TimelineEntityType.YARN_APPLICATION.toString(), null),
EnumSet.of(Field.ALL)); new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
assertNotNull(e1); assertNotNull(e1);
assertEquals(3, e1.getConfigs().size()); assertEquals(3, e1.getConfigs().size());
assertEquals(1, e1.getIsRelatedToEntities().size()); assertEquals(1, e1.getIsRelatedToEntities().size());
Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1", Set<TimelineEntity> es1 = reader.getEntities(
"some_flow_name", 1002345678919L, null, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
null, null, null, null, null, null, null, EnumSet.of(Field.ALL)); null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
assertEquals(3, es1.size()); assertEquals(3, es1.size());
} }
@Test @Test
public void testReadAppsDefaultView() throws Exception { public void testReadAppsDefaultView() throws Exception {
TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name", TimelineEntity e1 = reader.getEntity(
new TimelineReaderContext("cluster1", "user1", "some_flow_name",
1002345678919L, "application_1111111111_2222", 1002345678919L, "application_1111111111_2222",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null); TimelineEntityType.YARN_APPLICATION.toString(), null),
new TimelineDataToRetrieve());
assertNotNull(e1); assertNotNull(e1);
assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() && assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() && e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
e1.getRelatesToEntities().isEmpty()); e1.getRelatesToEntities().isEmpty());
Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1", Set<TimelineEntity> es1 = reader.getEntities(
"some_flow_name", 1002345678919L, null, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
null, null, null, null, null, null, null, null); null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve());
assertEquals(3, es1.size()); assertEquals(3, es1.size());
for (TimelineEntity e : es1) { for (TimelineEntity e : es1) {
assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() && assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
@ -1275,18 +1313,22 @@ public void testReadAppsDefaultView() throws Exception {
@Test @Test
public void testReadAppsByFields() throws Exception { public void testReadAppsByFields() throws Exception {
TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name", TimelineEntity e1 = reader.getEntity(
new TimelineReaderContext("cluster1", "user1", "some_flow_name",
1002345678919L, "application_1111111111_2222", 1002345678919L, "application_1111111111_2222",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, TimelineEntityType.YARN_APPLICATION.toString(), null),
EnumSet.of(Field.INFO, Field.CONFIGS)); new TimelineDataToRetrieve(
null, null, EnumSet.of(Field.INFO, Field.CONFIGS)));
assertNotNull(e1); assertNotNull(e1);
assertEquals(3, e1.getConfigs().size()); assertEquals(3, e1.getConfigs().size());
assertEquals(0, e1.getIsRelatedToEntities().size()); assertEquals(0, e1.getIsRelatedToEntities().size());
Set<TimelineEntity> es1 = reader.getEntities( Set<TimelineEntity> es1 = reader.getEntities(
"user1", "cluster1", "some_flow_name", 1002345678919L, null, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
null, null, null, null, null, null, null, null),
EnumSet.of(Field.IS_RELATED_TO, Field.METRICS)); new TimelineEntityFilters(),
new TimelineDataToRetrieve(
null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS)));
assertEquals(3, es1.size()); assertEquals(3, es1.size());
int metricsCnt = 0; int metricsCnt = 0;
int isRelatedToCnt = 0; int isRelatedToCnt = 0;
@ -1306,15 +1348,19 @@ public void testReadAppsConfigPrefix() throws Exception {
TimelineFilterList list = TimelineFilterList list =
new TimelineFilterList(Operator.OR, new TimelineFilterList(Operator.OR,
new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name", TimelineEntity e1 = reader.getEntity(
new TimelineReaderContext("cluster1", "user1", "some_flow_name",
1002345678919L, "application_1111111111_2222", 1002345678919L, "application_1111111111_2222",
TimelineEntityType.YARN_APPLICATION.toString(), null, list, null, null); TimelineEntityType.YARN_APPLICATION.toString(), null),
new TimelineDataToRetrieve(list, null, null));
assertNotNull(e1); assertNotNull(e1);
assertEquals(1, e1.getConfigs().size()); assertEquals(1, e1.getConfigs().size());
Set<TimelineEntity> es1 = reader.getEntities( Set<TimelineEntity> es1 = reader.getEntities(
"user1", "cluster1", "some_flow_name", 1002345678919L, null, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
null, null, null, null, null, list, null, null); null) ,
new TimelineEntityFilters(),
new TimelineDataToRetrieve(list, null, null));
int cfgCnt = 0; int cfgCnt = 0;
for (TimelineEntity entity : es1) { for (TimelineEntity entity : es1) {
cfgCnt += entity.getConfigs().size(); cfgCnt += entity.getConfigs().size();
@ -1329,9 +1375,12 @@ public void testReadAppsConfigFilterPrefix() throws Exception {
new TimelineFilterList(Operator.OR, new TimelineFilterList(Operator.OR,
new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
Set<TimelineEntity> entities = reader.getEntities( Set<TimelineEntity> entities = reader.getEntities(
"user1", "cluster1", "some_flow_name", 1002345678919L, null, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
null, null, confFilters, null, null, list, null, null); null),
new TimelineEntityFilters(null, null, null, null, null, null,
confFilters, null, null),
new TimelineDataToRetrieve(list, null, null));
assertEquals(1, entities.size()); assertEquals(1, entities.size());
int cfgCnt = 0; int cfgCnt = 0;
for (TimelineEntity entity : entities) { for (TimelineEntity entity : entities) {
@ -1345,15 +1394,19 @@ public void testReadAppsMetricPrefix() throws Exception {
TimelineFilterList list = TimelineFilterList list =
new TimelineFilterList(Operator.OR, new TimelineFilterList(Operator.OR,
new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name", TimelineEntity e1 = reader.getEntity(
new TimelineReaderContext("cluster1", "user1", "some_flow_name",
1002345678919L, "application_1111111111_2222", 1002345678919L, "application_1111111111_2222",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, list, null); TimelineEntityType.YARN_APPLICATION.toString(), null),
new TimelineDataToRetrieve(null, list, null));
assertNotNull(e1); assertNotNull(e1);
assertEquals(1, e1.getMetrics().size()); assertEquals(1, e1.getMetrics().size());
Set<TimelineEntity> es1 = reader.getEntities( Set<TimelineEntity> es1 = reader.getEntities(
"user1", "cluster1", "some_flow_name", 1002345678919L, null, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
null, null, null, null, null, null, list, null); null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve(null, list, null));
int metricCnt = 0; int metricCnt = 0;
for (TimelineEntity entity : es1) { for (TimelineEntity entity : es1) {
metricCnt += entity.getMetrics().size(); metricCnt += entity.getMetrics().size();
@ -1368,9 +1421,12 @@ public void testReadAppsMetricFilterPrefix() throws Exception {
new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
Set<String> metricFilters = ImmutableSet.of("MAP1_SLOT_MILLIS"); Set<String> metricFilters = ImmutableSet.of("MAP1_SLOT_MILLIS");
Set<TimelineEntity> entities = reader.getEntities( Set<TimelineEntity> entities = reader.getEntities(
"user1", "cluster1", "some_flow_name", 1002345678919L, null, new TimelineReaderContext("cluster1", "user1", "some_flow_name",
TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
null, null, null, metricFilters, null, null, list, null); null),
new TimelineEntityFilters(null, null, null, null, null, null, null,
metricFilters, null),
new TimelineDataToRetrieve(null, list, null));
int metricCnt = 0; int metricCnt = 0;
assertEquals(1, entities.size()); assertEquals(1, entities.size());
for (TimelineEntity entity : entities) { for (TimelineEntity entity : entities) {

View File

@ -45,6 +45,9 @@
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; 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.TimelineEntityType;
import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl; import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl;
import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl; import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
@ -179,10 +182,12 @@ public void testWriteFlowRunMinMax() throws Exception {
hbr.init(c1); hbr.init(c1);
hbr.start(); hbr.start();
// get the flow activity entity // get the flow activity entity
Set<TimelineEntity> entities = Set<TimelineEntity> entities = hbr.getEntities(
hbr.getEntities(null, cluster, null, null, null, new TimelineReaderContext(cluster, null, null, null, null,
TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null, TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
null, null, null, null, null, null, null, null, null); new TimelineEntityFilters(10L, null, null, null, null, null,
null, null, null),
new TimelineDataToRetrieve());
assertEquals(1, entities.size()); assertEquals(1, entities.size());
for (TimelineEntity e : entities) { for (TimelineEntity e : entities) {
FlowActivityEntity flowActivity = (FlowActivityEntity)e; FlowActivityEntity flowActivity = (FlowActivityEntity)e;
@ -235,10 +240,12 @@ public void testWriteFlowActivityOneFlow() throws Exception {
hbr.init(c1); hbr.init(c1);
hbr.start(); hbr.start();
Set<TimelineEntity> entities = Set<TimelineEntity> entities = hbr.getEntities(
hbr.getEntities(user, cluster, flow, null, null, new TimelineReaderContext(cluster, user, flow, null, null,
TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null, TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
null, null, null, null, null, null, null, null, null); new TimelineEntityFilters(10L, null, null, null, null, null,
null, null, null),
new TimelineDataToRetrieve());
assertEquals(1, entities.size()); assertEquals(1, entities.size());
for (TimelineEntity e : entities) { for (TimelineEntity e : entities) {
FlowActivityEntity entity = (FlowActivityEntity)e; FlowActivityEntity entity = (FlowActivityEntity)e;
@ -350,10 +357,12 @@ public void testFlowActivityTableOneFlowMultipleRunIds() throws IOException {
hbr.init(c1); hbr.init(c1);
hbr.start(); hbr.start();
Set<TimelineEntity> entities = Set<TimelineEntity> entities = hbr.getEntities(
hbr.getEntities(null, cluster, null, null, null, new TimelineReaderContext(cluster, null, null, null, null,
TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null, TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
null, null, null, null, null, null, null, null, null); new TimelineEntityFilters(10L, null, null, null, null, null,
null, null, null),
new TimelineDataToRetrieve());
assertEquals(1, entities.size()); assertEquals(1, entities.size());
for (TimelineEntity e : entities) { for (TimelineEntity e : entities) {
FlowActivityEntity flowActivity = (FlowActivityEntity)e; FlowActivityEntity flowActivity = (FlowActivityEntity)e;

View File

@ -44,7 +44,9 @@
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; 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.TimelineEntityType;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
@ -183,8 +185,10 @@ public void testWriteFlowRunMinMax() throws Exception {
hbr.init(c1); hbr.init(c1);
hbr.start(); hbr.start();
// get the flow run entity // get the flow run entity
TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null, TimelineEntity entity = hbr.getEntity(
TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, null); new TimelineReaderContext(cluster, user, flow, runid, null,
TimelineEntityType.YARN_FLOW_RUN.toString(), null),
new TimelineDataToRetrieve());
assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
FlowRunEntity flowRun = (FlowRunEntity)entity; FlowRunEntity flowRun = (FlowRunEntity)entity;
assertEquals(minStartTs, flowRun.getStartTime()); assertEquals(minStartTs, flowRun.getStartTime());
@ -242,8 +246,10 @@ public void testWriteFlowRunMetricsOneFlow() throws Exception {
hbr = new HBaseTimelineReaderImpl(); hbr = new HBaseTimelineReaderImpl();
hbr.init(c1); hbr.init(c1);
hbr.start(); hbr.start();
TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null, TimelineEntity entity = hbr.getEntity(
TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, null); new TimelineReaderContext(cluster, user, flow, runid, null,
TimelineEntityType.YARN_FLOW_RUN.toString(), null),
new TimelineDataToRetrieve());
assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
Set<TimelineMetric> metrics = entity.getMetrics(); Set<TimelineMetric> metrics = entity.getMetrics();
assertEquals(2, metrics.size()); assertEquals(2, metrics.size());
@ -350,9 +356,10 @@ public void testWriteFlowRunMetricsPrefix() throws Exception {
TimelineFilterList metricsToRetrieve = TimelineFilterList metricsToRetrieve =
new TimelineFilterList(new TimelinePrefixFilter(TimelineCompareOp.EQUAL, new TimelineFilterList(new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
metric1.substring(0, metric1.indexOf("_") + 1))); metric1.substring(0, metric1.indexOf("_") + 1)));
TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null, TimelineEntity entity = hbr.getEntity(
TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, new TimelineReaderContext(cluster, user, flow, runid, null,
metricsToRetrieve, null); TimelineEntityType.YARN_FLOW_RUN.toString(), null),
new TimelineDataToRetrieve(null, metricsToRetrieve, null));
assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
Set<TimelineMetric> metrics = entity.getMetrics(); Set<TimelineMetric> metrics = entity.getMetrics();
assertEquals(1, metrics.size()); assertEquals(1, metrics.size());
@ -373,9 +380,11 @@ public void testWriteFlowRunMetricsPrefix() throws Exception {
} }
} }
Set<TimelineEntity> entities = hbr.getEntities(user, cluster, flow, runid, Set<TimelineEntity> entities = hbr.getEntities(
null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, new TimelineReaderContext(cluster, user, flow, runid, null,
null, null, null, null, null, null, null, metricsToRetrieve, null); TimelineEntityType.YARN_FLOW_RUN.toString(), null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve(null, metricsToRetrieve, null));
assertEquals(1, entities.size()); assertEquals(1, entities.size());
for (TimelineEntity timelineEntity : entities) { for (TimelineEntity timelineEntity : entities) {
Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics(); Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics();
@ -441,18 +450,21 @@ public void testWriteFlowRunsMetricFields() throws Exception {
hbr = new HBaseTimelineReaderImpl(); hbr = new HBaseTimelineReaderImpl();
hbr.init(c1); hbr.init(c1);
hbr.start(); hbr.start();
Set<TimelineEntity> entities = hbr.getEntities(user, cluster, flow, runid, Set<TimelineEntity> entities = hbr.getEntities(
null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, new TimelineReaderContext(cluster, user, flow, runid, null,
null, null, null, null, null, null, null, null, null); TimelineEntityType.YARN_FLOW_RUN.toString(), null),
new TimelineEntityFilters(),
new TimelineDataToRetrieve());
assertEquals(1, entities.size()); assertEquals(1, entities.size());
for (TimelineEntity timelineEntity : entities) { for (TimelineEntity timelineEntity : entities) {
assertEquals(0, timelineEntity.getMetrics().size()); assertEquals(0, timelineEntity.getMetrics().size());
} }
entities = hbr.getEntities(user, cluster, flow, runid, entities = hbr.getEntities(
null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, new TimelineReaderContext(cluster, user, flow, runid, null,
null, null, null, null, null, null, null, null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
EnumSet.of(Field.METRICS)); new TimelineEntityFilters(),
new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
assertEquals(1, entities.size()); assertEquals(1, entities.size());
for (TimelineEntity timelineEntity : entities) { for (TimelineEntity timelineEntity : entities) {
Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics(); Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics();