From 9cb1287e9b8425f91de925f411c3c2a8fa9fe2a3 Mon Sep 17 00:00:00 2001 From: Sangjin Lee Date: Wed, 3 Feb 2016 16:03:55 -0800 Subject: [PATCH] YARN-4446. Refactor reader API for better extensibility (Varun Saxena via sjlee) --- .../reader/TimelineDataToRetrieve.java | 119 +++++ .../reader/TimelineEntityFilters.java | 187 ++++++++ .../reader/TimelineReaderContext.java | 10 + .../reader/TimelineReaderManager.java | 44 +- .../reader/TimelineReaderUtils.java | 4 +- .../reader/TimelineReaderWebServices.java | 417 ++++++++---------- .../TimelineReaderWebServicesUtils.java | 68 +++ .../storage/FileSystemTimelineReaderImpl.java | 120 +++-- .../storage/HBaseTimelineReaderImpl.java | 36 +- .../storage/TimelineReader.java | 234 +++++----- .../reader/ApplicationEntityReader.java | 204 ++++----- .../reader/FlowActivityEntityReader.java | 59 +-- .../storage/reader/FlowRunEntityReader.java | 101 ++--- .../storage/reader/GenericEntityReader.java | 192 ++++---- .../storage/reader/TimelineEntityReader.java | 101 ++--- .../reader/TimelineEntityReaderFactory.java | 74 ++-- .../TestFileSystemTimelineReaderImpl.java | 156 ++++--- .../storage/TestHBaseTimelineStorage.java | 252 +++++++---- .../flow/TestHBaseStorageFlowActivity.java | 33 +- .../storage/flow/TestHBaseStorageFlowRun.java | 48 +- 20 files changed, 1369 insertions(+), 1090 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java new file mode 100644 index 00000000000..0cc83d7ee53 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java @@ -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.
+ * Data to retrieve contains the following :
+ * + */ +@Private +@Unstable +public class TimelineDataToRetrieve { + private TimelineFilterList confsToRetrieve; + private TimelineFilterList metricsToRetrieve; + private EnumSet fieldsToRetrieve; + + public TimelineDataToRetrieve() { + this(null, null, null); + } + + public TimelineDataToRetrieve(TimelineFilterList confs, + TimelineFilterList metrics, EnumSet 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 getFieldsToRetrieve() { + return fieldsToRetrieve; + } + + public void setFieldsToRetrieve(EnumSet 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); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java new file mode 100644 index 00000000000..5b2c300c5db --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java @@ -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.
+ * Filters contain the following :
+ *
    + *
  • limit - A limit on the number of entities to return. If null + * or {@literal <=0}, defaults to {@link #DEFAULT_LIMIT}.
  • + *
  • createdTimeBegin - Matched entities should not be created + * before this timestamp. If null or {@literal <=0}, defaults to 0.
  • + *
  • createdTimeEnd - Matched entities should not be created after + * this timestamp. If null or {@literal <=0}, defaults to + * {@link Long#MAX_VALUE}.
  • + *
  • relatesTo - Matched entities should relate to given entities. + * If null or empty, the relations are not matched.
  • + *
  • isRelatedTo - Matched entities should be related to given + * entities. If null or empty, the relations are not matched.
  • + *
  • infoFilters - Matched entities should have exact matches to + * the given info represented as key-value pairs. If null or empty, the + * filter is not applied.
  • + *
  • configFilters - Matched entities should have exact matches to + * the given configs represented as key-value pairs. If null or empty, the + * filter is not applied.
  • + *
  • metricFilters - Matched entities should contain the given + * metrics. If null or empty, the filter is not applied.
  • + *
  • eventFilters - Matched entities should contain the given + * events. If null or empty, the filter is not applied.
  • + *
+ */ +@Private +@Unstable +public class TimelineEntityFilters { + private Long limit; + private Long createdTimeBegin; + private Long createdTimeEnd; + private Map> relatesTo; + private Map> isRelatedTo; + private Map infoFilters; + private Map configFilters; + private Set metricFilters; + private Set 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> entityRelatesTo, + Map> entityIsRelatedTo, + Map entityInfoFilters, + Map entityConfigFilters, + Set entityMetricFilters, + Set 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> getRelatesTo() { + return relatesTo; + } + + public void setRelatesTo(Map> relations) { + this.relatesTo = relations; + } + + public Map> getIsRelatedTo() { + return isRelatedTo; + } + + public void setIsRelatedTo(Map> relations) { + this.isRelatedTo = relations; + } + + public Map getInfoFilters() { + return infoFilters; + } + + public void setInfoFilters(Map filters) { + this.infoFilters = filters; + } + + public Map getConfigFilters() { + return configFilters; + } + + public void setConfigFilters(Map filters) { + this.configFilters = filters; + } + + public Set getMetricFilters() { + return metricFilters; + } + + public void setMetricFilters(Set filters) { + this.metricFilters = filters; + } + + public Set getEventFilters() { + return eventFilters; + } + + public void setEventFilters(Set filters) { + this.eventFilters = filters; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java index d0ad54a5419..633bb232e4f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java @@ -18,11 +18,15 @@ 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; /** * Encapsulates fields necessary to make a query in timeline reader. */ +@Private +@Unstable public class TimelineReaderContext extends TimelineContext { private String entityType; @@ -34,6 +38,12 @@ public TimelineReaderContext(String clusterId, String userId, String flowName, this.entityId = entityId; } + public TimelineReaderContext(TimelineReaderContext other) { + this(other.getClusterId(), other.getUserId(), other.getFlowName(), + other.getFlowRunId(), other.getAppId(), other.getEntityType(), + other.getEntityId()); + } + @Override public int hashCode() { final int prime = 31; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java index a7d864ef1f4..84a97eacd20 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java @@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.reader; import java.io.IOException; -import java.util.EnumSet; -import java.util.Map; import java.util.Set; 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.conf.YarnConfiguration; 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; @@ -122,23 +119,14 @@ private static void fillUID(TimelineEntityType entityType, * * @see TimelineReader#getEntities */ - public Set getEntities(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - EnumSet fieldsToRetrieve) throws IOException { - String cluster = getClusterID(clusterId, getConfig()); - Set entities = - reader.getEntities(userId, cluster, flowName, flowRunId, appId, - entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo, - isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters, - null, null, fieldsToRetrieve); + public Set getEntities(TimelineReaderContext context, + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { + context.setClusterId(getClusterID(context.getClusterId(), getConfig())); + Set entities = reader.getEntities( + new TimelineReaderContext(context), filters, dataToRetrieve); if (entities != null) { - TimelineEntityType type = getTimelineEntityType(entityType); - TimelineReaderContext context = new TimelineReaderContext(cluster, userId, - flowName, flowRunId, appId, entityType, null); + TimelineEntityType type = getTimelineEntityType(context.getEntityType()); for (TimelineEntity entity : entities) { fillUID(type, entity, context); } @@ -152,18 +140,14 @@ public Set getEntities(String userId, String clusterId, * * @see TimelineReader#getEntity */ - public TimelineEntity getEntity(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - String entityId, EnumSet fields) throws IOException { - String cluster = getClusterID(clusterId, getConfig()); - TimelineEntity entity = - reader.getEntity(userId, cluster, flowName, flowRunId, appId, - entityType, entityId, null, null, fields); - + public TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException { + context.setClusterId( + getClusterID(context.getClusterId(), getConfig())); + TimelineEntity entity = reader.getEntity( + new TimelineReaderContext(context), dataToRetrieve); if (entity != null) { - TimelineEntityType type = getTimelineEntityType(entityType); - TimelineReaderContext context = new TimelineReaderContext(cluster, userId, - flowName, flowRunId, appId, entityType, null); + TimelineEntityType type = getTimelineEntityType(context.getEntityType()); fillUID(type, entity, context); } return entity; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java index 66abbfcda81..c93c631b364 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java @@ -90,7 +90,7 @@ static List split(final String str, final char delimiterChar, // String needs to be split here. Copy remaining chars and add the // string to list. 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. startOffset = ++offset; builder = new StringBuilder(len - offset); @@ -103,7 +103,7 @@ static List split(final String str, final char delimiterChar, builder.append(str.substring(startOffset)); } // Add the last part of delimited string to list. - list.add(builder.toString()); + list.add(builder.toString().trim()); return list; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java index 7a70de87f9c..d68199a550e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java @@ -46,8 +46,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.security.UserGroupInformation; 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.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; @@ -70,8 +68,6 @@ public class TimelineReaderWebServices { @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 RANGE_DELIMITER = "-"; private static final String DATE_PATTERN = "yyyyMMdd"; @@ -243,15 +239,15 @@ public TimelineAbout about( * events. This is represented as eventfilters=eventid1, eventid2... * @param fields Specifies which fields of the entity object to retrieve, see * {@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). * * @return If successful, a HTTP 200(OK) response having a JSON representing - * a set of {@link TimelineEntity} instances of the given entity type is - * returned. - * On failures, + * a set of TimelineEntity instances of the given entity type + * is returned.
+ * On failures,
* If any problem occurs in parsing request or UID is incorrect, - * HTTP 400(Bad Request) is returned. + * HTTP 400(Bad Request) is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -290,30 +286,14 @@ public Set getEntities( if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } - entities = timelineReaderManager.getEntities( - TimelineReaderWebServicesUtils.parseStr(context.getUserId()), - TimelineReaderWebServicesUtils.parseStr(context.getClusterId()), - TimelineReaderWebServicesUtils.parseStr(context.getFlowName()), - context.getFlowRunId(), - TimelineReaderWebServicesUtils.parseStr(context.getAppId()), - TimelineReaderWebServicesUtils.parseStr(entityType), - 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)); + context.setEntityType( + TimelineReaderWebServicesUtils.parseStr(entityType)); + entities = timelineReaderManager.getEntities(context, + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit or flowrunid"); @@ -374,17 +354,17 @@ public Set getEntities( * events. This is represented as eventfilters=eventid1, eventid2... * @param fields Specifies which fields of the entity object to retrieve, see * {@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). * * @return If successful, a HTTP 200(OK) response having a JSON representing - * a set of {@link TimelineEntity} instances of the given entity type is - * returned. - * On failures, + * a set of TimelineEntity instances of the given entity type + * is returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* If flow context information cannot be retrieved, HTTP 404(Not Found) - * is returned. + * is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -463,17 +443,17 @@ public Set getEntities( * events. This is represented as eventfilters=eventid1, eventid2... * @param fields Specifies which fields of the entity object to retrieve, see * {@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). * * @return If successful, a HTTP 200(OK) response having a JSON representing - * a set of {@link TimelineEntity} instances of the given entity type is - * returned. - * On failures, + * a set of TimelineEntity instances of the given entity type + * is returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* If flow context information cannot be retrieved, HTTP 404(Not Found) - * is returned. + * is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -512,29 +492,14 @@ public Set getEntities( Set entities = null; try { entities = timelineReaderManager.getEntities( - TimelineReaderWebServicesUtils.parseStr(userId), - TimelineReaderWebServicesUtils.parseStr(clusterId), - TimelineReaderWebServicesUtils.parseStr(flowName), - TimelineReaderWebServicesUtils.parseLongStr(flowRunId), - TimelineReaderWebServicesUtils.parseStr(appId), - TimelineReaderWebServicesUtils.parseStr(entityType), - 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)); + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, flowRunId, appId, entityType, null), + + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit or flowrunid"); @@ -559,16 +524,16 @@ public Set getEntities( * UID and then used to query backend(Mandatory path param). * @param fields Specifies which fields of the entity object to retrieve, see * {@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). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * {@link TimelineEntity} instance is returned. - * On failures, + * TimelineEntity instance is returned.
+ * On failures,
* If any problem occurs in parsing request or UID is incorrect, - * HTTP 400(Bad Request) is returned. + * HTTP 400(Bad Request) is returned.
* If entity for the given entity id cannot be found, HTTP 404(Not Found) - * is returned. + * is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -597,11 +562,9 @@ public TimelineEntity getEntity( if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } - entity = timelineReaderManager.getEntity(context.getUserId(), - context.getClusterId(), context.getFlowName(), context.getFlowRunId(), - context.getAppId(), context.getEntityType(), context.getEntityId(), - TimelineReaderWebServicesUtils.parseFieldsStr( - fields, COMMA_DELIMITER)); + entity = timelineReaderManager.getEntity(context, + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -638,16 +601,16 @@ public TimelineEntity getEntity( * param). * @param fields Specifies which fields of the entity object to retrieve, see * {@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). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * {@link TimelineEntity} instance is returned. - * On failures, + * TimelineEntity instance is returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* 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.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -690,16 +653,16 @@ public TimelineEntity getEntity( * param). * @param fields Specifies which fields of the entity object to retrieve, see * {@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). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * {@link TimelineEntity} instance is returned. - * On failures, + * TimelineEntity instance is returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* 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.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -728,17 +691,12 @@ public TimelineEntity getEntity( init(res); TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); TimelineEntity entity = null; - String type = TimelineReaderWebServicesUtils.parseStr(entityType); - String id = TimelineReaderWebServicesUtils.parseStr(entityId); try { entity = timelineReaderManager.getEntity( - TimelineReaderWebServicesUtils.parseStr(userId), - TimelineReaderWebServicesUtils.parseStr(clusterId), - TimelineReaderWebServicesUtils.parseStr(flowName), - TimelineReaderWebServicesUtils.parseLongStr(flowRunId), - TimelineReaderWebServicesUtils.parseStr(appId), type, id, - TimelineReaderWebServicesUtils.parseFieldsStr( - fields, COMMA_DELIMITER)); + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, flowRunId, appId, entityType, entityId), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -746,8 +704,8 @@ public TimelineEntity getEntity( if (entity == null) { LOG.info("Processed URL " + url + " but entity not found" + " (Took " + (endTime - startTime) + " ms.)"); - throw new NotFoundException("Timeline entity {id: " + id + ", type: " + - type + " } is not found"); + throw new NotFoundException("Timeline entity {id: " + entityId + + ", type: " + entityType + " } is not found"); } LOG.info("Processed URL " + url + " (Took " + (endTime - startTime) + " ms.)"); @@ -765,13 +723,13 @@ public TimelineEntity getEntity( * (Mandatory path param). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * {@link FlowRunEntity} instance is returned. By default, all metrics for - * the flow run will be returned. - * On failures, + * FlowRunEntity instance is returned. By default, all + * metrics for the flow run will be returned.
+ * On failures,
* If any problem occurs in parsing request or UID is incorrect, - * HTTP 400(Bad Request) is returned. + * HTTP 400(Bad Request) is returned.
* If flow run for the given flow run id cannot be found, HTTP 404 - * (Not Found) is returned. + * (Not Found) is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -799,9 +757,9 @@ public TimelineEntity getFlowRun( if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } - entity = timelineReaderManager.getEntity(context.getUserId(), - context.getClusterId(), context.getFlowName(), context.getFlowRunId(), - null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null); + context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString()); + entity = timelineReaderManager.getEntity(context, + new TimelineDataToRetrieve()); } catch (Exception e) { 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * {@link FlowRunEntity} instance is returned. By default, all metrics for - * the flow run will be returned. - * On failures, + * FlowRunEntity instance is returned. By default, all + * metrics for the flow run will be returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* If flow run for the given flow run id cannot be found, HTTP 404 - * (Not Found) is returned. + * (Not Found) is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -865,13 +823,13 @@ public TimelineEntity getFlowRun( * @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 - * {@link FlowRunEntity} instance is returned. By default, all metrics for - * the flow run will be returned. - * On failures, + * FlowRunEntity instance is returned. By default, all + * metrics for the flow run will be returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* If flow run for the given flow run id cannot be found, HTTP 404 - * (Not Found) is returned. + * (Not Found) is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -899,11 +857,10 @@ public TimelineEntity getFlowRun( TimelineEntity entity = null; try { entity = timelineReaderManager.getEntity( - TimelineReaderWebServicesUtils.parseStr(userId), - TimelineReaderWebServicesUtils.parseStr(clusterId), - TimelineReaderWebServicesUtils.parseStr(flowName), - TimelineReaderWebServicesUtils.parseLongStr(flowRunId), - null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null); + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, flowRunId, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineDataToRetrieve()); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -930,8 +887,6 @@ public TimelineEntity getFlowRun( * @param uId a delimited string containing clusterid, userid, and flow name * which are extracted from UID and then used to query backend(Mandatory * 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 createdTimeStart If specified, matched flow runs should not be * created before this timestamp(Optional query param). @@ -943,10 +898,11 @@ public TimelineEntity getFlowRun( * other than metrics are returned(Optional query param). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * set of {@link FlowRunEntity} instances for the given flow are returned. - * On failures, + * set of FlowRunEntity instances for the given flow are + * returned.
+ * On failures,
* If any problem occurs in parsing request or UID is incorrect, - * HTTP 400(Bad Request) is returned. + * HTTP 400(Bad Request) is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -978,14 +934,13 @@ public Set getFlowRuns( if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } - entities = timelineReaderManager.getEntities(context.getUserId(), - context.getClusterId(), context.getFlowName(), null, null, - TimelineEntityType.YARN_FLOW_RUN.toString(), - TimelineReaderWebServicesUtils.parseLongStr(limit), - TimelineReaderWebServicesUtils.parseLongStr(createdTimeStart), - TimelineReaderWebServicesUtils.parseLongStr(createdTimeEnd), - null, null, null, null, null, null, TimelineReaderWebServicesUtils. - parseFieldsStr(fields, COMMA_DELIMITER)); + context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString()); + entities = timelineReaderManager.getEntities(context, + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, null, null, null, + null, null, null), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit"); } @@ -1019,10 +974,11 @@ public Set getFlowRuns( * other than metrics are returned(Optional query param). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * set of {@link FlowRunEntity} instances for the given flow are returned. - * On failures, + * set of FlowRunEntity instances for the given flow are + * returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1064,10 +1020,11 @@ public Set getFlowRuns( * other than metrics are returned(Optional query param). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * set of {@link FlowRunEntity} instances for the given flow are returned. - * On failures, + * set of FlowRunEntity instances for the given flow are + * returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1097,15 +1054,14 @@ public Set getFlowRuns( Set entities = null; try { entities = timelineReaderManager.getEntities( - TimelineReaderWebServicesUtils.parseStr(userId), - TimelineReaderWebServicesUtils.parseStr(clusterId), - TimelineReaderWebServicesUtils.parseStr(flowName), null, null, - TimelineEntityType.YARN_FLOW_RUN.toString(), - TimelineReaderWebServicesUtils.parseLongStr(limit), - TimelineReaderWebServicesUtils.parseLongStr(createdTimeStart), - TimelineReaderWebServicesUtils.parseLongStr(createdTimeEnd), - null, null, null, null, null, null, TimelineReaderWebServicesUtils. - parseFieldsStr(fields, COMMA_DELIMITER)); + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, null, null, null, + null, null, null), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit"); } @@ -1142,12 +1098,12 @@ public Set getFlowRuns( * "daterange=-20150711" returns flows active on and before 20150711. * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * set of {@link FlowActivityEntity} instances are returned. - * On failures, + * set of FlowActivityEntity instances are returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server - * Error) is returned. + * Error) is returned.
*/ @GET @Path("/flows/") @@ -1185,10 +1141,10 @@ public Set getFlows( * "daterange=-20150711" returns flows active on and before 20150711. * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * set of {@link FlowActivityEntity} instances are returned. - * On failures, + * set of FlowActivityEntity instances are returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1214,11 +1170,17 @@ public Set getFlows( Set entities = null; try { 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( - null, TimelineReaderWebServicesUtils.parseStr(clusterId), null, null, - null, TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), - TimelineReaderWebServicesUtils.parseLongStr(limit), range.dateStart, - range.dateEnd, null, null, null, null, null, null, null); + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, null, null, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null), + entityFilters, TimelineReaderWebServicesUtils. + createTimelineDataToRetrieve(null, null, null)); } catch (Exception e) { handleException(e, url, startTime, "limit"); } @@ -1242,16 +1204,16 @@ public Set getFlows( * backend(Mandatory path param). * @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 - * 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * {@link TimelineEntity} instance is returned. - * On failures, + * TimelineEntity instance is returned.
+ * On failures,
* If any problem occurs in parsing request or UID is incorrect, - * HTTP 400(Bad Request) is returned. + * HTTP 400(Bad Request) is returned.
* If app for the given app id cannot be found, HTTP 404(Not Found) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1280,11 +1242,10 @@ public TimelineEntity getApp( if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } - entity = timelineReaderManager.getEntity(context.getUserId(), - context.getClusterId(), context.getFlowName(), context.getFlowRunId(), - context.getAppId(), TimelineEntityType.YARN_APPLICATION.toString(), - null, TimelineReaderWebServicesUtils.parseFieldsStr( - fields, COMMA_DELIMITER)); + context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString()); + entity = timelineReaderManager.getEntity(context, + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { 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 fields Specifies which fields of the app entity object to retrieve, * 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * {@link TimelineEntity} instance is returned. - * On failures, + * TimelineEntity instance is returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* 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.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1361,16 +1322,16 @@ public TimelineEntity getApp( * @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, * 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing a - * {@link TimelineEntity} instance is returned. - * On failures, + * TimelineEntity instance is returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* 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.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1399,14 +1360,11 @@ public TimelineEntity getApp( TimelineEntity entity = null; try { entity = timelineReaderManager.getEntity( - TimelineReaderWebServicesUtils.parseStr(userId), - TimelineReaderWebServicesUtils.parseStr(clusterId), - TimelineReaderWebServicesUtils.parseStr(flowName), - TimelineReaderWebServicesUtils.parseLongStr(flowRunId), - TimelineReaderWebServicesUtils.parseStr(appId), - TimelineEntityType.YARN_APPLICATION.toString(), null, - TimelineReaderWebServicesUtils.parseFieldsStr( - fields, COMMA_DELIMITER)); + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, flowRunId, appId, + TimelineEntityType.YARN_APPLICATION.toString(), null), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -1459,15 +1417,15 @@ public TimelineEntity getApp( * events. This is represented as eventfilters=eventid1, eventid2... * @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 - * 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing - * a set of {@link TimelineEntity} instances representing apps is - * returned. - * On failures, + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
* If any problem occurs in parsing request or UID is incorrect, - * HTTP 400(Bad Request) is returned. + * HTTP 400(Bad Request) is returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1505,30 +1463,13 @@ public Set getFlowRunApps( if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } - entities = timelineReaderManager.getEntities( - TimelineReaderWebServicesUtils.parseStr(context.getUserId()), - TimelineReaderWebServicesUtils.parseStr(context.getClusterId()), - TimelineReaderWebServicesUtils.parseStr(context.getFlowName()), - context.getFlowRunId(), - TimelineReaderWebServicesUtils.parseStr(context.getAppId()), - TimelineEntityType.YARN_APPLICATION.toString(), - 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)); + context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString()); + entities = timelineReaderManager.getEntities(context, + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, null, fields)); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit or flowrunid"); @@ -1582,15 +1523,15 @@ public Set getFlowRunApps( * events. This is represented as eventfilters=eventid1, eventid2... * @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 - * 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing - * a set of {@link TimelineEntity} instances representing apps is - * returned. - * On failures, + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1661,15 +1602,15 @@ public Set getFlowRunApps( * events. This is represented as eventfilters=eventid1, eventid2... * @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 - * 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing - * a set of {@link TimelineEntity} instances representing apps is - * returned. - * On failures, + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1739,15 +1680,15 @@ public Set getFlowRunApps( * events. This is represented as eventfilters=eventid1, eventid2... * @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 - * 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing - * a set of {@link TimelineEntity} instances representing apps is - * returned. - * On failures, + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ @@ -1815,15 +1756,15 @@ public Set getFlowApps( * events. This is represented as eventfilters=eventid1, eventid2... * @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 - * 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). * * @return If successful, a HTTP 200(OK) response having a JSON representing - * a set of {@link TimelineEntity} instances representing apps is - * returned. - * On failures, + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
* If any problem occurs in parsing request, HTTP 400(Bad Request) is - * returned. + * returned.
* For all other errors while retrieving data, HTTP 500(Internal Server * Error) is returned. */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java index d052d5163ea..fc07e51c1cc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java @@ -35,9 +35,77 @@ * Set of utility methods to be used by timeline reader web services. */ final class TimelineReaderWebServicesUtils { + private static final String COMMA_DELIMITER = ","; + private static final String COLON_DELIMITER = ":"; + 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 * instance, if delimiter is ",", then the string should be represented as diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java index 72c92494eb5..aa0071f5aec 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java @@ -44,7 +44,9 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; 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.webapp.YarnJacksonJaxbJsonProvider; import org.codehaus.jackson.JsonGenerationException; @@ -264,22 +266,8 @@ private static TimelineEntity readEntityFromFile(BufferedReader reader) } private Set getEntities(File dir, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet 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; - } - + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { // First sort the selected entities based on created/start time. Map> sortedEntities = new TreeMap<>( @@ -303,41 +291,48 @@ public int compare(Long l1, Long l2) { if (!entity.getType().equals(entityType)) { continue; } - if (!isTimeInRange(entity.getCreatedTime(), createdTimeBegin, - createdTimeEnd)) { + if (!isTimeInRange(entity.getCreatedTime(), + filters.getCreatedTimeBegin(), filters.getCreatedTimeEnd())) { continue; } - if (relatesTo != null && !relatesTo.isEmpty() && - !TimelineStorageUtils - .matchRelations(entity.getRelatesToEntities(), relatesTo)) { + if (filters.getRelatesTo() != null && + !filters.getRelatesTo().isEmpty() && + !TimelineStorageUtils.matchRelations( + entity.getRelatesToEntities(), filters.getRelatesTo())) { continue; } - if (isRelatedTo != null && !isRelatedTo.isEmpty() && - !TimelineStorageUtils - .matchRelations(entity.getIsRelatedToEntities(), isRelatedTo)) { + if (filters.getIsRelatedTo() != null && + !filters.getIsRelatedTo().isEmpty() && + !TimelineStorageUtils.matchRelations( + entity.getIsRelatedToEntities(), filters.getIsRelatedTo())) { continue; } - if (infoFilters != null && !infoFilters.isEmpty() && - !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) { - continue; - } - if (configFilters != null && !configFilters.isEmpty() && + if (filters.getInfoFilters() != null && + !filters.getInfoFilters().isEmpty() && !TimelineStorageUtils.matchFilters( - entity.getConfigs(), configFilters)) { + entity.getInfo(), filters.getInfoFilters())) { 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( - entity.getMetrics(), metricFilters)) { + entity.getMetrics(), filters.getMetricFilters())) { continue; } - if (eventFilters != null && !eventFilters.isEmpty() && + if (filters.getEventFilters() != null && + !filters.getEventFilters().isEmpty() && !TimelineStorageUtils.matchEventFilters( - entity.getEvents(), eventFilters)) { + entity.getEvents(), filters.getEventFilters())) { continue; } - TimelineEntity entityToBeReturned = - createEntityToBeReturned(entity, fieldsToRetrieve); + TimelineEntity entityToBeReturned = createEntityToBeReturned( + entity, dataToRetrieve.getFieldsToRetrieve()); Set entitiesCreatedAtSameTime = sortedEntities.get(entityToBeReturned.getCreatedTime()); if (entitiesCreatedAtSameTime == null) { @@ -355,7 +350,7 @@ public int compare(Long l1, Long l2) { for (TimelineEntity entity : entitySet) { entities.add(entity); ++entitiesAdded; - if (entitiesAdded >= limit) { + if (entitiesAdded >= filters.getLimit()) { return entities; } } @@ -371,45 +366,40 @@ public void serviceInit(Configuration conf) throws Exception { } @Override - public TimelineEntity getEntity(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - String entityId, TimelineFilterList confsToRetrieve, - TimelineFilterList metricsToRetrieve, EnumSet fieldsToRetrieve) - throws IOException { - String flowRunPath = getFlowRunPath(userId, clusterId, flowName, - flowRunId, appId); + public TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException { + String flowRunPath = getFlowRunPath(context.getUserId(), + context.getClusterId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId()); File dir = new File(new File(rootPath, ENTITIES_DIR), - clusterId + "/" + flowRunPath + "/" + appId + "/" + entityType); - File entityFile = - new File(dir, entityId + TIMELINE_SERVICE_STORAGE_EXTENSION); + context.getClusterId() + "/" + flowRunPath + "/" + context.getAppId() + + "/" + context.getEntityType()); + File entityFile = new File( + dir, context.getEntityId() + TIMELINE_SERVICE_STORAGE_EXTENSION); try (BufferedReader reader = new BufferedReader(new InputStreamReader( new FileInputStream(entityFile), Charset.forName("UTF-8")))) { TimelineEntity entity = readEntityFromFile(reader); - return createEntityToBeReturned(entity, fieldsToRetrieve); + return createEntityToBeReturned( + entity, dataToRetrieve.getFieldsToRetrieve()); } catch (FileNotFoundException e) { - LOG.info("Cannot find entity {id:" + entityId + " , type:" + entityType + - "}. Will send HTTP 404 in response."); + LOG.info("Cannot find entity {id:" + context.getEntityId() + " , type:" + + context.getEntityType() + "}. Will send HTTP 404 in response."); return null; } } @Override - public Set getEntities(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet fieldsToRetrieve) throws IOException { - String flowRunPath = - getFlowRunPath(userId, clusterId, flowName, flowRunId, appId); + public Set getEntities(TimelineReaderContext context, + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { + String flowRunPath = getFlowRunPath(context.getUserId(), + context.getClusterId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId()); File dir = new File(new File(rootPath, ENTITIES_DIR), - clusterId + "/" + flowRunPath + "/" + appId + "/" + entityType); - return getEntities(dir, entityType, limit, createdTimeBegin, createdTimeEnd, - relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters, - eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve); + context.getClusterId() + "/" + flowRunPath + "/" + + context.getAppId() + "/" + context.getEntityType()); + return getEntities(dir, context.getEntityType(), filters, dataToRetrieve); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java index 0ce9a9407c5..9f8257f257c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java @@ -19,8 +19,6 @@ import java.io.IOException; -import java.util.EnumSet; -import java.util.Map; import java.util.Set; import org.apache.commons.logging.Log; @@ -31,7 +29,9 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.service.AbstractService; 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.TimelineEntityReaderFactory; @@ -65,33 +65,21 @@ protected void serviceStop() throws Exception { } @Override - public TimelineEntity getEntity(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - String entityId, TimelineFilterList confsToRetrieve, - TimelineFilterList metricsToRetrieve, EnumSet fieldsToRetrieve) - throws IOException { + public TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException { TimelineEntityReader reader = - TimelineEntityReaderFactory.createSingleEntityReader(userId, clusterId, - flowName, flowRunId, appId, entityType, entityId, confsToRetrieve, - metricsToRetrieve, fieldsToRetrieve); + TimelineEntityReaderFactory.createSingleEntityReader(context, + dataToRetrieve); return reader.readEntity(hbaseConf, conn); } @Override - public Set getEntities(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet fieldsToRetrieve) throws IOException { + public Set getEntities(TimelineReaderContext context, + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { TimelineEntityReader reader = - TimelineEntityReaderFactory.createMultipleEntitiesReader(userId, - clusterId, flowName, flowRunId, appId, entityType, limit, - createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo, - infoFilters, configFilters, metricFilters, eventFilters, - confsToRetrieve, metricsToRetrieve, fieldsToRetrieve); + TimelineEntityReaderFactory.createMultipleEntitiesReader(context, + filters, dataToRetrieve); return reader.readEntities(hbaseConf, conn); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java index 0eeb195641d..ccb33b79ca4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java @@ -20,28 +20,20 @@ import java.io.IOException; -import java.util.EnumSet; -import java.util.Map; import java.util.Set; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; 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.server.timelineservice.reader.filter.TimelineFilterList; -import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; +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; /** ATSv2 reader interface. */ @Private @Unstable public interface TimelineReader extends Service { - /** - * Default limit for {@link #getEntities}. - */ - long DEFAULT_LIMIT = 100; - /** * Possible fields to retrieve for {@link #getEntities} and * {@link #getEntity}. @@ -57,55 +49,61 @@ public enum Field { } /** - *

The API to fetch the single entity given the entity identifier in the - * scope of the given context.

- * - * @param userId - * Context user Id(optional). - * @param clusterId - * Context cluster Id(mandatory). - * @param flowName - * Context flow Id (optional). - * @param flowRunId - * Context flow run Id (optional). - * @param appId - * Context app Id (mandatory) - * @param entityType - * Entity type (mandatory) - * @param entityId - * Entity Id (mandatory) - * @param confsToRetrieve - * 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 to return in response. - * @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 to return in response. - * @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 {@link TimelineEntity} instance or null. The entity will - * contain the metadata plus the given fields to retrieve. + *

The API to fetch the single entity given the identifier(depending on + * the entity type) in the scope of the given context.

+ * @param context Context which defines the scope in which query has to be + * made. Use getters of {@link TimelineReaderContext} to fetch context + * fields. Context contains the following :
+ *
    + *
  • entityType - Entity type(mandatory).
  • + *
  • clusterId - Identifies the cluster(mandatory).
  • + *
  • userId - Identifies the user.
  • + *
  • flowName - Context flow name.
  • + *
  • flowRunId - Context flow run id.
  • + *
  • appId - Context app id.
  • + *
  • entityId - Entity id.
  • + *
+ * Fields in context which are mandatory depends on entity type. Entity + * type is always mandatory. In addition to entity type, below is the list + * of context fields which are mandatory, based on entity type.
+ *
    + *
  • If entity type is YARN_FLOW_RUN (i.e. query to fetch a specific flow + * run), clusterId, userId, flowName and flowRunId are mandatory.
  • + *
  • If entity type is YARN_APPLICATION (i.e. query to fetch a specific + * app), query is within the scope of clusterId, userId, flowName, + * flowRunId and appId. But out of this, only clusterId and appId are + * mandatory. If only clusterId and appId are supplied, backend storage + * must fetch the flow context information i.e. userId, flowName and + * flowRunId first and based on that, fetch the app. If flow context + * information is also given, app can be directly fetched. + *
  • + *
  • For other entity types (i.e. query to fetch generic entity), query + * is within the scope of clusterId, userId, flowName, flowRunId, appId, + * entityType and entityId. But out of this, only clusterId, appId, + * entityType and entityId are mandatory. If flow context information is + * not supplied, backend storage must fetch the flow context information + * i.e. userId, flowName and flowRunId first and based on that, fetch the + * entity. If flow context information is also given, entity can be + * directly queried. + *
  • + *
+ * @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 TimelineEntity instance or null. The entity will + * contain the metadata plus the given fields to retrieve.
* If entityType is YARN_FLOW_RUN, entity returned is of type - * {@link FlowRunEntity}. + * FlowRunEntity.
* For all other entity types, entity returned is of type - * {@link TimelineEntity}. + * TimelineEntity. * @throws IOException */ - TimelineEntity getEntity(String userId, String clusterId, String flowName, - Long flowRunId, String appId, String entityType, String entityId, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet fieldsToRetrieve) throws IOException; + TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException; /** - *

The API to search for a set of entities of the given the entity type in + *

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 * predicates include the created time window, limit to number of entities to * 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 * entries, the qualified entity needs to meet all or them.

* - * @param userId - * Context user Id(optional). - * @param clusterId - * Context cluster Id(mandatory). - * @param flowName - * Context flow Id (optional). - * @param flowRunId - * Context flow run Id (optional). - * @param appId - * Context app Id (mandatory) - * @param entityType - * Entity type (mandatory) - * @param limit - * A limit on the number of entities to return (optional). If null or <=0, - * defaults to {@link #DEFAULT_LIMIT}. - * @param createdTimeBegin - * Matched entities should not be created before this timestamp (optional). - * If null or <=0, defaults to 0. - * @param createdTimeEnd - * Matched entities should not be created after this timestamp (optional). - * If null or <=0, defaults to {@link Long#MAX_VALUE}. - * @param relatesTo - * Matched entities should relate to given entities (optional). - * @param isRelatedTo - * Matched entities should be related to given entities (optional). - * @param infoFilters - * Matched entities should have exact matches to the given info represented - * as key-value pairs (optional). If null or empty, the filter is not - * applied. - * @param configFilters - * Matched entities should have exact matches to the given configs - * represented as key-value pairs (optional). If null or empty, the filter - * is not applied. - * @param metricFilters - * Matched entities should contain the given metrics (optional). If null - * or empty, the filter is not applied. - * @param eventFilters - * Matched entities should contain the given events (optional). If null - * or empty, the filter is not applied. - * @param confsToRetrieve - * 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. This should not be - * confused with configFilters which is used to decide which entities to - * return instead. - * @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 + * @param context Context which defines the scope in which query has to be + * made. Use getters of {@link TimelineReaderContext} to fetch context + * fields. Context contains the following :
+ *
    + *
  • entityType - Entity type(mandatory).
  • + *
  • clusterId - Identifies the cluster(mandatory).
  • + *
  • userId - Identifies the user.
  • + *
  • flowName - Context flow name.
  • + *
  • flowRunId - Context flow run id.
  • + *
  • appId - Context app id.
  • + *
+ * Although entityId is also part of context, it has no meaning for + * getEntities.
+ * Fields in context which are mandatory depends on entity type. Entity + * type is always mandatory. In addition to entity type, below is the list + * of context fields which are mandatory, based on entity type.
+ *
    + *
  • If entity type is YARN_FLOW_ACTIVITY (i.e. query to fetch flows), + * only clusterId is mandatory. + *
  • + *
  • If entity type is YARN_FLOW_RUN (i.e. query to fetch flow runs), + * clusterId, userId and flowName are mandatory.
  • + *
  • If entity type is YARN_APPLICATION (i.e. query to fetch apps), we + * can either get all apps within the context of flow name or within the + * context of flow run. If apps are queried within the scope of flow name, + * clusterId, userId and flowName are supplied. If they are queried within + * the scope of flow run, clusterId, userId, flowName and flowRunId are + * supplied.
  • + *
  • For other entity types (i.e. query to fetch generic entities), query + * is within the scope of clusterId, userId, flowName, flowRunId, appId and + * entityType. But out of this, only clusterId, appId and entityType are + * mandatory. If flow context information is not supplied, backend storage + * must fetch the flow context information i.e. userId, flowName and + * flowRunId first and based on that, fetch the entities. If flow context + * information is also given, entities can be directly queried. + *
  • + *
+ * @param filters Specifies filters which restrict the number of entities + * to return. Use getters of TimelineEntityFilters class to fetch + * various filters. All the filters are optional. Refer to + * {@link TimelineEntityFilters} for details. + * @param dataToRetrieve Specifies which data to retrieve for each entity. Use + * getters of TimelineDataToRetrieve class to fetch dataToRetrieve + * fields. All the dataToRetrieve fields are optional. Refer to + * {@link TimelineDataToRetrieve} for details. + * @return A set of 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 * metadata(id, type and created time) plus the given fields to retrieve. + *
* If entityType is YARN_FLOW_ACTIVITY, entities returned are of type - * {@link FlowActivityEntity}. + * FlowActivityEntity.
* If entityType is YARN_FLOW_RUN, entities returned are of type - * {@link FlowRunEntity}. + * FlowRunEntity.
* For all other entity types, entities returned are of type - * {@link TimelineEntity}. + * TimelineEntity. * @throws IOException */ - Set getEntities(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet fieldsToRetrieve) throws IOException; + Set getEntities( + TimelineReaderContext context, + TimelineEntityFilters filters, + TimelineDataToRetrieve dataToRetrieve) throws IOException; } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java index 4d61076ba27..387f7d72716 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java @@ -19,8 +19,6 @@ 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.hbase.client.Connection; @@ -38,9 +36,10 @@ import org.apache.hadoop.hbase.filter.QualifierFilter; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; 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.storage.TimelineReader; 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.ApplicationColumnFamily; @@ -60,26 +59,14 @@ class ApplicationEntityReader extends GenericEntityReader { private static final ApplicationTable APPLICATION_TABLE = new ApplicationTable(); - public ApplicationEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet fieldsToRetrieve) { - super(userId, clusterId, flowName, flowRunId, appId, entityType, limit, - createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo, infoFilters, - configFilters, metricFilters, eventFilters, confsToRetrieve, - metricsToRetrieve, fieldsToRetrieve, true); + public ApplicationEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve, true); } - public ApplicationEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - String entityId, TimelineFilterList confsToRetrieve, - TimelineFilterList metricsToRetrieve, EnumSet fieldsToRetrieve) { - super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId, - confsToRetrieve, metricsToRetrieve, fieldsToRetrieve); + public ApplicationEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); } /** @@ -92,12 +79,13 @@ protected BaseTable getTable() { @Override protected FilterList constructFilterListBasedOnFields() { FilterList list = new FilterList(Operator.MUST_PASS_ONE); + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); // Fetch all the columns. - if (fieldsToRetrieve.contains(Field.ALL) && - (confsToRetrieve == null || - confsToRetrieve.getFilterList().isEmpty()) && - (metricsToRetrieve == null || - metricsToRetrieve.getFilterList().isEmpty())) { + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (dataToRetrieve.getConfsToRetrieve() == null || + dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) && + (dataToRetrieve.getMetricsToRetrieve() == null || + dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) { return list; } FilterList infoColFamilyList = new FilterList(); @@ -107,61 +95,70 @@ protected FilterList constructFilterListBasedOnFields() { new BinaryComparator(ApplicationColumnFamily.INFO.getBytes())); infoColFamilyList.addFilter(infoColumnFamily); // Events not required. - if (!fieldsToRetrieve.contains(Field.EVENTS) && - !fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) { + TimelineEntityFilters filters = getFilters(); + if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.EVENTS) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (singleEntityRead || filters.getEventFilters() == null)) { infoColFamilyList.addFilter( new QualifierFilter(CompareOp.NOT_EQUAL, new BinaryPrefixComparator( ApplicationColumnPrefix.EVENT.getColumnPrefixBytes("")))); } // info not required. - if (!fieldsToRetrieve.contains(Field.INFO) && - !fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) { + if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.INFO) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (singleEntityRead || filters.getInfoFilters() == null)) { infoColFamilyList.addFilter( new QualifierFilter(CompareOp.NOT_EQUAL, new BinaryPrefixComparator( ApplicationColumnPrefix.INFO.getColumnPrefixBytes("")))); } // is releated to not required. - if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) && - !fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) { + if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.IS_RELATED_TO) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (singleEntityRead || filters.getIsRelatedTo() == null)) { infoColFamilyList.addFilter( new QualifierFilter(CompareOp.NOT_EQUAL, new BinaryPrefixComparator( ApplicationColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes("")))); } // relates to not required. - if (!fieldsToRetrieve.contains(Field.RELATES_TO) && - !fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) { + if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.RELATES_TO) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (singleEntityRead || filters.getRelatesTo() == null)) { infoColFamilyList.addFilter( new QualifierFilter(CompareOp.NOT_EQUAL, new BinaryPrefixComparator( ApplicationColumnPrefix.RELATES_TO.getColumnPrefixBytes("")))); } list.addFilter(infoColFamilyList); - if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) || - (confsToRetrieve != null && - !confsToRetrieve.getFilterList().isEmpty())) { + if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS) || + (!singleEntityRead && filters.getConfigFilters() != null)) || + (dataToRetrieve.getConfsToRetrieve() != null && + !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty())) { FilterList filterCfg = new FilterList(new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(ApplicationColumnFamily.CONFIGS.getBytes()))); - if (confsToRetrieve != null && - !confsToRetrieve.getFilterList().isEmpty()) { + if (dataToRetrieve.getConfsToRetrieve() != null && + !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) { filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList( - ApplicationColumnPrefix.CONFIG, confsToRetrieve)); + ApplicationColumnPrefix.CONFIG, + dataToRetrieve.getConfsToRetrieve())); } list.addFilter(filterCfg); } - if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) || - (metricsToRetrieve != null && - !metricsToRetrieve.getFilterList().isEmpty())) { + if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) || + (!singleEntityRead && filters.getMetricFilters() != null)) || + (dataToRetrieve.getMetricsToRetrieve() != null && + !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) { FilterList filterMetrics = new FilterList(new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(ApplicationColumnFamily.METRICS.getBytes()))); - if (metricsToRetrieve != null && - !metricsToRetrieve.getFilterList().isEmpty()) { + if (dataToRetrieve.getMetricsToRetrieve() != null && + !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) { filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList( - ApplicationColumnPrefix.METRIC, metricsToRetrieve)); + ApplicationColumnPrefix.METRIC, + dataToRetrieve.getMetricsToRetrieve())); } list.addFilter(filterMetrics); } @@ -171,9 +168,10 @@ protected FilterList constructFilterListBasedOnFields() { @Override protected Result getResult(Configuration hbaseConf, Connection conn, FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); byte[] rowKey = - ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId, - appId); + ApplicationRowKey.getRowKey(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId(), context.getAppId()); Get get = new Get(rowKey); get.setMaxVersions(Integer.MAX_VALUE); if (filterList != null && !filterList.getFilters().isEmpty()) { @@ -184,66 +182,54 @@ protected Result getResult(Configuration hbaseConf, Connection conn, @Override protected void validateParams() { - Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null"); - Preconditions.checkNotNull(entityType, "entityType shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getEntityType(), + "entityType shouldn't be null"); if (singleEntityRead) { - Preconditions.checkNotNull(appId, "appId shouldn't be null"); + Preconditions.checkNotNull(getContext().getAppId(), + "appId shouldn't be null"); } else { - Preconditions.checkNotNull(userId, "userId 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"); } } @Override protected void augmentParams(Configuration hbaseConf, Connection conn) throws IOException { + TimelineReaderContext context = getContext(); if (singleEntityRead) { - if (flowName == null || flowRunId == null || userId == null) { - FlowContext context = - lookupFlowContext(clusterId, appId, hbaseConf, conn); - flowName = context.flowName; - flowRunId = context.flowRunId; - userId = context.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; + if (context.getFlowName() == null || context.getFlowRunId() == null || + context.getUserId() == null) { + FlowContext flowContext = lookupFlowContext( + context.getClusterId(), context.getAppId(), hbaseConf, conn); + context.setFlowName(flowContext.flowName); + context.setFlowRunId(flowContext.flowRunId); + context.setUserId(flowContext.userId); } } + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); } @Override protected ResultScanner getResults(Configuration hbaseConf, Connection conn, FilterList filterList) throws IOException { Scan scan = new Scan(); - if (flowRunId != null) { + TimelineReaderContext context = getContext(); + if (context.getFlowRunId() != null) { scan.setRowPrefixFilter(ApplicationRowKey. - getRowKeyPrefix(clusterId, userId, flowName, flowRunId)); + getRowKeyPrefix(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId())); } else { scan.setRowPrefixFilter(ApplicationRowKey. - getRowKeyPrefix(clusterId, userId, flowName)); + getRowKeyPrefix(context.getClusterId(), context.getUserId(), + context.getFlowName())); } FilterList newList = new FilterList(); - newList.addFilter(new PageFilter(limit)); + newList.addFilter(new PageFilter(getFilters().getLimit())); if (filterList != null && !filterList.getFilters().isEmpty()) { newList.addFilter(filterList); } @@ -261,23 +247,27 @@ protected TimelineEntity parseEntity(Result result) throws IOException { String entityId = ApplicationColumn.ID.readResult(result).toString(); entity.setId(entityId); + TimelineEntityFilters filters = getFilters(); // fetch created time Number createdTime = (Number)ApplicationColumn.CREATED_TIME.readResult(result); entity.setCreatedTime(createdTime.longValue()); - if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin || - entity.getCreatedTime() > createdTimeEnd)) { + if (!singleEntityRead && + (entity.getCreatedTime() < filters.getCreatedTimeBegin() || + entity.getCreatedTime() > filters.getCreatedTimeEnd())) { return null; } - + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); // 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) || fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) { readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO, true); if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations( - entity.getIsRelatedToEntities(), isRelatedTo)) { + entity.getIsRelatedToEntities(), filters.getIsRelatedTo())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -287,13 +277,15 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) { readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO, false); if (checkRelatesTo && !TimelineStorageUtils.matchRelations( - entity.getRelatesToEntities(), relatesTo)) { + entity.getRelatesToEntities(), filters.getRelatesTo())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -303,12 +295,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.INFO) || checkInfo) { readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false); if (checkInfo && - !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) { + !TimelineStorageUtils.matchFilters( + entity.getInfo(), filters.getInfoFilters())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -318,12 +312,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) { readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true); if (checkConfigs && !TimelineStorageUtils.matchFilters( - entity.getConfigs(), configFilters)) { + entity.getConfigs(), filters.getConfigFilters())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -333,12 +329,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) { readEvents(entity, result, true); if (checkEvents && !TimelineStorageUtils.matchEventFilters( - entity.getEvents(), eventFilters)) { + entity.getEvents(), filters.getEventFilters())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -348,12 +346,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) { readMetrics(entity, result, ApplicationColumnPrefix.METRIC); if (checkMetrics && !TimelineStorageUtils.matchMetricFilters( - entity.getMetrics(), metricFilters)) { + entity.getMetrics(), filters.getMetricFilters())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java index 048f608c734..96350da2c3d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java @@ -18,9 +18,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.reader; 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.hbase.client.Connection; @@ -32,8 +30,9 @@ 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.server.timelineservice.storage.TimelineReader; -import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +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.BaseTable; import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix; 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 = new FlowActivityTable(); - public FlowActivityEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - EnumSet fieldsToRetrieve) { - super(userId, clusterId, flowName, flowRunId, appId, entityType, limit, - createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo, infoFilters, - configFilters, metricFilters, eventFilters, null, null, - fieldsToRetrieve, true); + public FlowActivityEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve, true); } - public FlowActivityEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - String entityId, EnumSet fieldsToRetrieve) { - super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId, - null, null, fieldsToRetrieve); + public FlowActivityEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); } /** @@ -79,21 +68,13 @@ protected BaseTable getTable() { @Override protected void validateParams() { - Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); } @Override protected void augmentParams(Configuration hbaseConf, Connection conn) 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 @@ -112,20 +93,24 @@ protected Result getResult(Configuration hbaseConf, Connection conn, protected ResultScanner getResults(Configuration hbaseConf, Connection conn, FilterList filterList) throws IOException { Scan scan = new Scan(); - if (createdTimeBegin == DEFAULT_BEGIN_TIME && - createdTimeEnd == DEFAULT_END_TIME) { + String clusterId = getContext().getClusterId(); + if (getFilters().getCreatedTimeBegin() == 0L && + getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) { + // All records have to be chosen. scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId)); } else { scan.setStartRow( - FlowActivityRowKey.getRowKeyPrefix(clusterId, createdTimeEnd)); + FlowActivityRowKey.getRowKeyPrefix(clusterId, + getFilters().getCreatedTimeEnd())); scan.setStopRow( 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 // the scanner may still return more than the limit; therefore we need to // read the right number as we iterate - scan.setFilter(new PageFilter(limit)); + scan.setFilter(new PageFilter(getFilters().getLimit())); return table.getResultScanner(hbaseConf, conn, scan); } @@ -137,8 +122,8 @@ protected TimelineEntity parseEntity(Result result) throws IOException { String user = rowKey.getUserId(); String flowName = rowKey.getFlowName(); - FlowActivityEntity flowActivity = - new FlowActivityEntity(clusterId, time, user, flowName); + FlowActivityEntity flowActivity = new FlowActivityEntity( + getContext().getClusterId(), time, user, flowName); // set the id flowActivity.setId(flowActivity.getId()); // get the list of run ids along with the version that are associated with diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java index 4f50b02a2f5..2d1c41cd9ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java @@ -18,9 +18,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.reader; 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.hbase.client.Connection; @@ -38,9 +35,10 @@ 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.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.storage.TimelineReader; 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.flow.FlowRunColumn; @@ -58,26 +56,14 @@ class FlowRunEntityReader extends TimelineEntityReader { private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable(); - public FlowRunEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet fieldsToRetrieve) { - super(userId, clusterId, flowName, flowRunId, appId, entityType, limit, - createdTimeBegin, createdTimeEnd, relatesTo, isRelatedTo, infoFilters, - configFilters, metricFilters, eventFilters, null, metricsToRetrieve, - fieldsToRetrieve, true); + public FlowRunEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve, true); } - public FlowRunEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - String entityId, TimelineFilterList confsToRetrieve, - TimelineFilterList metricsToRetrieve, EnumSet fieldsToRetrieve) { - super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId, - null, metricsToRetrieve, fieldsToRetrieve); + public FlowRunEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); } /** @@ -90,35 +76,21 @@ protected BaseTable getTable() { @Override protected void validateParams() { - Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null"); - Preconditions.checkNotNull(userId, "userId shouldn't be null"); - Preconditions.checkNotNull(flowName, "flowName shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getUserId(), + "userId shouldn't be null"); + Preconditions.checkNotNull(getContext().getFlowName(), + "flowName shouldn't be null"); if (singleEntityRead) { - Preconditions.checkNotNull(flowRunId, "flowRunId shouldn't be null"); + Preconditions.checkNotNull(getContext().getFlowRunId(), + "flowRunId shouldn't be null"); } } @Override protected void augmentParams(Configuration hbaseConf, Connection conn) { - if (!singleEntityRead) { - 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); - } - } + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); } @Override @@ -129,9 +101,11 @@ protected FilterList constructFilterListBasedOnFields() { FamilyFilter infoColumnFamily = new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(FlowRunColumnFamily.INFO.getBytes())); + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); // Metrics not required. - if (!singleEntityRead && !fieldsToRetrieve.contains(Field.METRICS) && - !fieldsToRetrieve.contains(Field.ALL)) { + if (!singleEntityRead && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL)) { FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE); infoColFamilyList.addFilter(infoColumnFamily); infoColFamilyList.addFilter( @@ -140,12 +114,12 @@ protected FilterList constructFilterListBasedOnFields() { FlowRunColumnPrefix.METRIC.getColumnPrefixBytes("")))); list.addFilter(infoColFamilyList); } - if (metricsToRetrieve != null && - !metricsToRetrieve.getFilterList().isEmpty()) { + if (dataToRetrieve.getMetricsToRetrieve() != null && + !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) { FilterList infoColFamilyList = new FilterList(); infoColFamilyList.addFilter(infoColumnFamily); infoColFamilyList.addFilter(TimelineFilterUtils.createHBaseFilterList( - FlowRunColumnPrefix.METRIC, metricsToRetrieve)); + FlowRunColumnPrefix.METRIC, dataToRetrieve.getMetricsToRetrieve())); list.addFilter(infoColFamilyList); } return list; @@ -154,8 +128,10 @@ protected FilterList constructFilterListBasedOnFields() { @Override protected Result getResult(Configuration hbaseConf, Connection conn, FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); byte[] rowKey = - FlowRunRowKey.getRowKey(clusterId, userId, flowName, flowRunId); + FlowRunRowKey.getRowKey(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId()); Get get = new Get(rowKey); get.setMaxVersions(Integer.MAX_VALUE); if (filterList != null && !filterList.getFilters().isEmpty()) { @@ -168,10 +144,12 @@ protected Result getResult(Configuration hbaseConf, Connection conn, protected ResultScanner getResults(Configuration hbaseConf, Connection conn, FilterList filterList) throws IOException { Scan scan = new Scan(); + TimelineReaderContext context = getContext(); scan.setRowPrefixFilter( - FlowRunRowKey.getRowKeyPrefix(clusterId, userId, flowName)); + FlowRunRowKey.getRowKeyPrefix(context.getClusterId(), + context.getUserId(), context.getFlowName())); FilterList newList = new FilterList(); - newList.addFilter(new PageFilter(limit)); + newList.addFilter(new PageFilter(getFilters().getLimit())); if (filterList != null && !filterList.getFilters().isEmpty()) { newList.addFilter(filterList); } @@ -181,11 +159,12 @@ protected ResultScanner getResults(Configuration hbaseConf, @Override protected TimelineEntity parseEntity(Result result) throws IOException { + TimelineReaderContext context = getContext(); FlowRunEntity flowRun = new FlowRunEntity(); - flowRun.setUser(userId); - flowRun.setName(flowName); + flowRun.setUser(context.getUserId()); + flowRun.setName(context.getFlowName()); if (singleEntityRead) { - flowRun.setRunId(flowRunId); + flowRun.setRunId(context.getFlowRunId()); } else { FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow()); flowRun.setRunId(rowKey.getFlowRunId()); @@ -196,8 +175,9 @@ protected TimelineEntity parseEntity(Result result) throws IOException { if (startTime != null) { flowRun.setStartTime(startTime.longValue()); } - if (!singleEntityRead && (flowRun.getStartTime() < createdTimeBegin || - flowRun.getStartTime() > createdTimeEnd)) { + if (!singleEntityRead && + (flowRun.getStartTime() < getFilters().getCreatedTimeBegin() || + flowRun.getStartTime() > getFilters().getCreatedTimeEnd())) { return null; } @@ -214,7 +194,8 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // read metrics - if (singleEntityRead || fieldsToRetrieve.contains(Field.METRICS)) { + if (singleEntityRead || + getDataToRetrieve().getFieldsToRetrieve().contains(Field.METRICS)) { readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java index 237b9acc9ec..3bc2f3f5999 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java @@ -42,9 +42,10 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; 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.storage.TimelineReader; 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.ApplicationTable; @@ -77,26 +78,15 @@ class GenericEntityReader extends TimelineEntityReader { */ private final AppToFlowTable appToFlowTable = new AppToFlowTable(); - public GenericEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet 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(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve, + boolean sortedKeys) { + super(ctxt, entityFilters, toRetrieve, sortedKeys); } - public GenericEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - String entityId, TimelineFilterList confsToRetrieve, - TimelineFilterList metricsToRetrieve, EnumSet fieldsToRetrieve) { - super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId, - confsToRetrieve, metricsToRetrieve, fieldsToRetrieve); + public GenericEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); } /** @@ -109,12 +99,13 @@ protected BaseTable getTable() { @Override protected FilterList constructFilterListBasedOnFields() { FilterList list = new FilterList(Operator.MUST_PASS_ONE); + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); // Fetch all the columns. - if (fieldsToRetrieve.contains(Field.ALL) && - (confsToRetrieve == null || - confsToRetrieve.getFilterList().isEmpty()) && - (metricsToRetrieve == null || - metricsToRetrieve.getFilterList().isEmpty())) { + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (dataToRetrieve.getConfsToRetrieve() == null || + dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) && + (dataToRetrieve.getMetricsToRetrieve() == null || + dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) { return list; } FilterList infoColFamilyList = new FilterList(); @@ -123,62 +114,69 @@ protected FilterList constructFilterListBasedOnFields() { new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(EntityColumnFamily.INFO.getBytes())); infoColFamilyList.addFilter(infoColumnFamily); + TimelineEntityFilters filters = getFilters(); // Events not required. - if (!fieldsToRetrieve.contains(Field.EVENTS) && - !fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) { + if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.EVENTS) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (singleEntityRead || filters.getEventFilters() == null)) { infoColFamilyList.addFilter( new QualifierFilter(CompareOp.NOT_EQUAL, new BinaryPrefixComparator( EntityColumnPrefix.EVENT.getColumnPrefixBytes("")))); } // info not required. - if (!fieldsToRetrieve.contains(Field.INFO) && - !fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) { + if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.INFO) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (singleEntityRead || filters.getInfoFilters() == null)) { infoColFamilyList.addFilter( new QualifierFilter(CompareOp.NOT_EQUAL, new BinaryPrefixComparator( EntityColumnPrefix.INFO.getColumnPrefixBytes("")))); } // is related to not required. - if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) && - !fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) { + if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.IS_RELATED_TO) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (singleEntityRead || filters.getIsRelatedTo() == null)) { infoColFamilyList.addFilter( new QualifierFilter(CompareOp.NOT_EQUAL, new BinaryPrefixComparator( EntityColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes("")))); } // relates to not required. - if (!fieldsToRetrieve.contains(Field.RELATES_TO) && - !fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) { + if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.RELATES_TO) && + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) && + (singleEntityRead || filters.getRelatesTo() == null)) { infoColFamilyList.addFilter( new QualifierFilter(CompareOp.NOT_EQUAL, new BinaryPrefixComparator( EntityColumnPrefix.RELATES_TO.getColumnPrefixBytes("")))); } list.addFilter(infoColFamilyList); - if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) || - (confsToRetrieve != null && - !confsToRetrieve.getFilterList().isEmpty())) { + if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS) || + (!singleEntityRead && filters.getConfigFilters() != null)) || + (dataToRetrieve.getConfsToRetrieve() != null && + !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty())) { FilterList filterCfg = new FilterList(new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(EntityColumnFamily.CONFIGS.getBytes()))); - if (confsToRetrieve != null && - !confsToRetrieve.getFilterList().isEmpty()) { + if (dataToRetrieve.getConfsToRetrieve() != null && + !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) { filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList( - EntityColumnPrefix.CONFIG, confsToRetrieve)); + EntityColumnPrefix.CONFIG, dataToRetrieve.getConfsToRetrieve())); } list.addFilter(filterCfg); } - if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) || - (metricsToRetrieve != null && - !metricsToRetrieve.getFilterList().isEmpty())) { + if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) || + (!singleEntityRead && filters.getMetricFilters() != null)) || + (dataToRetrieve.getMetricsToRetrieve() != null && + !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) { FilterList filterMetrics = new FilterList(new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(EntityColumnFamily.METRICS.getBytes()))); - if (metricsToRetrieve != null && - !metricsToRetrieve.getFilterList().isEmpty()) { + if (dataToRetrieve.getMetricsToRetrieve() != null && + !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) { filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList( - EntityColumnPrefix.METRIC, metricsToRetrieve)); + EntityColumnPrefix.METRIC, dataToRetrieve.getMetricsToRetrieve())); } list.addFilter(filterMetrics); } @@ -215,56 +213,42 @@ public FlowContext(String user, String flowName, Long flowRunId) { @Override protected void validateParams() { - Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null"); - Preconditions.checkNotNull(appId, "appId shouldn't be null"); - Preconditions.checkNotNull(entityType, "entityType shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getAppId(), + "appId shouldn't be null"); + Preconditions.checkNotNull(getContext().getEntityType(), + "entityType shouldn't be null"); if (singleEntityRead) { - Preconditions.checkNotNull(entityId, "entityId shouldn't be null"); + Preconditions.checkNotNull(getContext().getEntityId(), + "entityId shouldn't be null"); } } @Override protected void augmentParams(Configuration hbaseConf, Connection conn) throws IOException { + TimelineReaderContext context = getContext(); // In reality all three should be null or neither should be null - if (flowName == null || flowRunId == null || userId == null) { - FlowContext context = - lookupFlowContext(clusterId, appId, hbaseConf, conn); - flowName = context.flowName; - flowRunId = context.flowRunId; - userId = context.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; - } + if (context.getFlowName() == null || context.getFlowRunId() == null || + context.getUserId() == null) { + FlowContext flowContext = lookupFlowContext( + context.getClusterId(), context.getAppId(), hbaseConf, conn); + context.setFlowName(flowContext.flowName); + context.setFlowRunId(flowContext.flowRunId); + context.setUserId(flowContext.userId); } + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); } @Override protected Result getResult(Configuration hbaseConf, Connection conn, FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); byte[] rowKey = - EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId, - entityType, entityId); + EntityRowKey.getRowKey(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId(), context.getAppId(), + context.getEntityType(), context.getEntityId()); Get get = new Get(rowKey); get.setMaxVersions(Integer.MAX_VALUE); 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 // and one type Scan scan = new Scan(); + TimelineReaderContext context = getContext(); 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); if (filterList != null && !filterList.getFilters().isEmpty()) { scan.setFilter(filterList); @@ -299,21 +285,25 @@ protected TimelineEntity parseEntity(Result result) throws IOException { String entityId = EntityColumn.ID.readResult(result).toString(); entity.setId(entityId); + TimelineEntityFilters filters = getFilters(); // fetch created time Number createdTime = (Number)EntityColumn.CREATED_TIME.readResult(result); entity.setCreatedTime(createdTime.longValue()); - if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin || - entity.getCreatedTime() > createdTimeEnd)) { + if (!singleEntityRead && + (entity.getCreatedTime() < filters.getCreatedTimeBegin() || + entity.getCreatedTime() > filters.getCreatedTimeEnd())) { return null; } - + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); // 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) || fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) { readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true); if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations( - entity.getIsRelatedToEntities(), isRelatedTo)) { + entity.getIsRelatedToEntities(), filters.getIsRelatedTo())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -323,12 +313,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) { readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false); if (checkRelatesTo && !TimelineStorageUtils.matchRelations( - entity.getRelatesToEntities(), relatesTo)) { + entity.getRelatesToEntities(), filters.getRelatesTo())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -338,12 +330,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.INFO) || checkInfo) { readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false); if (checkInfo && - !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) { + !TimelineStorageUtils.matchFilters( + entity.getInfo(), filters.getInfoFilters())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -353,12 +347,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) { readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true); if (checkConfigs && !TimelineStorageUtils.matchFilters( - entity.getConfigs(), configFilters)) { + entity.getConfigs(), filters.getConfigFilters())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -368,12 +364,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) { readEvents(entity, result, false); if (checkEvents && !TimelineStorageUtils.matchEventFilters( - entity.getEvents(), eventFilters)) { + entity.getEvents(), filters.getEventFilters())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && @@ -383,12 +381,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException { } // 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) || fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) { readMetrics(entity, result, EntityColumnPrefix.METRIC); if (checkMetrics && !TimelineStorageUtils.matchMetricFilters( - entity.getMetrics(), metricFilters)) { + entity.getMetrics(), filters.getMetricFilters())) { return null; } if (!fieldsToRetrieve.contains(Field.ALL) && diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java index bc86b6d007d..454c1792f63 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.reader; import java.io.IOException; -import java.util.EnumSet; import java.util.Map; import java.util.NavigableMap; import java.util.NavigableSet; @@ -34,8 +33,9 @@ import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; 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.storage.TimelineReader.Field; +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.BaseTable; import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; @@ -46,32 +46,12 @@ */ public abstract class TimelineEntityReader { 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 String userId; - protected String clusterId; - protected String flowName; - protected Long flowRunId; - protected String appId; - protected String entityType; - protected EnumSet fieldsToRetrieve; - // used only for a single entity read mode - protected String entityId; + private TimelineReaderContext context; + private TimelineDataToRetrieve dataToRetrieve; // used only for multiple entity read mode - protected Long limit; - protected Long createdTimeBegin; - protected Long createdTimeEnd; - protected Map> relatesTo; - protected Map> isRelatedTo; - protected Map infoFilters; - protected Map configFilters; - protected Set metricFilters; - protected Set eventFilters; - protected TimelineFilterList confsToRetrieve; - protected TimelineFilterList metricsToRetrieve; + private TimelineEntityFilters filters; /** * Main table the entity reader uses. @@ -89,34 +69,14 @@ public abstract class TimelineEntityReader { /** * Instantiates a reader for multiple-entity reads. */ - protected TimelineEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve, - EnumSet fieldsToRetrieve, boolean sortedKeys) { + protected TimelineEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve, + boolean sortedKeys) { this.singleEntityRead = false; this.sortedKeys = sortedKeys; - this.userId = userId; - this.clusterId = clusterId; - this.flowName = flowName; - 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.context = ctxt; + this.dataToRetrieve = toRetrieve; + this.filters = entityFilters; this.table = getTable(); } @@ -124,21 +84,11 @@ protected TimelineEntityReader(String userId, String clusterId, /** * Instantiates a reader for single-entity reads. */ - protected TimelineEntityReader(String userId, String clusterId, - String flowName, Long flowRunId, String appId, String entityType, - String entityId, TimelineFilterList confsToRetrieve, - TimelineFilterList metricsToRetrieve, EnumSet fieldsToRetrieve) { + protected TimelineEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { this.singleEntityRead = true; - this.userId = userId; - this.clusterId = clusterId; - 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.context = ctxt; + this.dataToRetrieve = toRetrieve; this.table = getTable(); } @@ -151,6 +101,18 @@ protected TimelineEntityReader(String userId, String clusterId, */ 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. */ @@ -163,7 +125,8 @@ public TimelineEntity readEntity(Configuration hbaseConf, Connection conn) Result result = getResult(hbaseConf, conn, filterList); if (result == null || result.isEmpty()) { // 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 parseEntity(result); @@ -190,11 +153,11 @@ public Set readEntities(Configuration hbaseConf, } entities.add(entity); if (!sortedKeys) { - if (entities.size() > limit) { + if (entities.size() > filters.getLimit()) { entities.pollLast(); } } else { - if (entities.size() == limit) { + if (entities.size() == filters.getLimit()) { break; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java index 2e2c652cfe3..f2bdacd38b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java @@ -17,13 +17,10 @@ */ 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.server.timelineservice.reader.filter.TimelineFilterList; -import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +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; /** * 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 * the specified input. */ - public static TimelineEntityReader createSingleEntityReader(String userId, - String clusterId, String flowName, Long flowRunId, String appId, - String entityType, String entityId, TimelineFilterList confs, - TimelineFilterList metrics, EnumSet fieldsToRetrieve) { + public static TimelineEntityReader createSingleEntityReader( + TimelineReaderContext context, TimelineDataToRetrieve dataToRetrieve) { // currently the types that are handled separate from the generic entity // table are application, flow run, and flow activity entities - if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) { - return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId, - appId, entityType, entityId, confs, metrics, fieldsToRetrieve); - } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) { - return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId, - appId, entityType, entityId, confs, metrics, fieldsToRetrieve); - } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) { - return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId, - appId, entityType, entityId, fieldsToRetrieve); + if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) { + return new ApplicationEntityReader(context, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_RUN.matches(context.getEntityType())) { + return new FlowRunEntityReader(context, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_ACTIVITY.matches(context.getEntityType())) { + return new FlowActivityEntityReader(context, dataToRetrieve); } else { // assume we're dealing with a generic entity read - return new GenericEntityReader(userId, clusterId, flowName, flowRunId, - appId, entityType, entityId, confs, metrics, fieldsToRetrieve); + return new GenericEntityReader(context, dataToRetrieve); } } @@ -59,37 +52,22 @@ public static TimelineEntityReader createSingleEntityReader(String userId, * Creates a timeline entity reader instance for reading set of entities with * the specified input and predicates. */ - public static TimelineEntityReader createMultipleEntitiesReader(String userId, - String clusterId, String flowName, Long flowRunId, String appId, - String entityType, Long limit, Long createdTimeBegin, Long createdTimeEnd, - Map> relatesTo, Map> isRelatedTo, - Map infoFilters, Map configFilters, - Set metricFilters, Set eventFilters, - TimelineFilterList confs, TimelineFilterList metrics, - EnumSet fieldsToRetrieve) { + public static TimelineEntityReader createMultipleEntitiesReader( + TimelineReaderContext context, TimelineEntityFilters filters, + TimelineDataToRetrieve dataToRetrieve) { // currently the types that are handled separate from the generic entity // table are application, flow run, and flow activity entities - if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) { - return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId, - appId, entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo, - isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters, - confs, metrics, fieldsToRetrieve); - } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) { - return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId, - appId, entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo, - 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); + if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) { + return new ApplicationEntityReader(context, filters, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_ACTIVITY.matches(context.getEntityType())) { + return new FlowActivityEntityReader(context, filters, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_RUN.matches(context.getEntityType())) { + return new FlowRunEntityReader(context, filters, dataToRetrieve); } else { // assume we're dealing with a generic entity read - return new GenericEntityReader(userId, clusterId, flowName, flowRunId, - appId, entityType, limit, createdTimeBegin, createdTimeEnd, relatesTo, - isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters, - confs, metrics, fieldsToRetrieve, false); + return new GenericEntityReader(context, filters, dataToRetrieve, false); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java index b7804e71c6e..a8a2ff81dfb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java @@ -37,6 +37,9 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; 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.util.timeline.TimelineUtils; import org.junit.AfterClass; @@ -258,9 +261,10 @@ public TimelineReader getTimelineReader() { public void testGetEntityDefaultView() throws Exception { // If no fields are specified, entity is returned with default view i.e. // only the id, type and created time. - TimelineEntity result = - reader.getEntity("user1", "cluster1", "flow1", 1L, "app1", - "app", "id_1", null, null, null); + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", "id_1"), + new TimelineDataToRetrieve(null, null, null)); Assert.assertEquals( (new TimelineEntity.Identifier("app", "id_1")).toString(), result.getIdentifier().toString()); @@ -272,9 +276,10 @@ public void testGetEntityDefaultView() throws Exception { @Test public void testGetEntityByClusterAndApp() throws Exception { // Cluster and AppId should be enough to get an entity. - TimelineEntity result = - reader.getEntity(null, "cluster1", null, null, "app1", - "app", "id_1", null, null, null); + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1", null, null, null, "app1", "app", + "id_1"), + new TimelineDataToRetrieve(null, null, null)); Assert.assertEquals( (new TimelineEntity.Identifier("app", "id_1")).toString(), result.getIdentifier().toString()); @@ -288,9 +293,10 @@ public void testGetEntityByClusterAndApp() throws Exception { public void testAppFlowMappingCsv() throws Exception { // Test getting an entity by cluster and app where flow entry // in app flow mapping csv has commas. - TimelineEntity result = - reader.getEntity(null, "cluster1", null, null, "app2", - "app", "id_5", null, null, null); + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1", null, null, null, "app2", + "app", "id_5"), + new TimelineDataToRetrieve(null, null, null)); Assert.assertEquals( (new TimelineEntity.Identifier("app", "id_5")).toString(), result.getIdentifier().toString()); @@ -300,10 +306,11 @@ public void testAppFlowMappingCsv() throws Exception { @Test public void testGetEntityCustomFields() throws Exception { // Specified fields in addition to default view will be returned. - TimelineEntity result = - reader.getEntity("user1", "cluster1", "flow1", 1L, - "app1", "app", "id_1", null, null, - EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS)); + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1","user1", "flow1", 1L, "app1", + "app", "id_1"), + new TimelineDataToRetrieve(null, null, + EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS))); Assert.assertEquals( (new TimelineEntity.Identifier("app", "id_1")).toString(), result.getIdentifier().toString()); @@ -318,9 +325,10 @@ public void testGetEntityCustomFields() throws Exception { @Test public void testGetEntityAllFields() throws Exception { // All fields of TimelineEntity will be returned. - TimelineEntity result = - reader.getEntity("user1", "cluster1", "flow1", 1L, "app1", "app", - "id_1", null, null, EnumSet.of(Field.ALL)); + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1","user1", "flow1", 1L, "app1", + "app", "id_1"), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); Assert.assertEquals( (new TimelineEntity.Identifier("app", "id_1")).toString(), result.getIdentifier().toString()); @@ -333,20 +341,21 @@ public void testGetEntityAllFields() throws Exception { @Test public void testGetAllEntities() throws Exception { - Set result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, null, null, null, null, null, null, null, null, null, null, - null); + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), new TimelineEntityFilters(), + new TimelineDataToRetrieve()); // All 3 entities will be returned Assert.assertEquals(4, result.size()); } @Test public void testGetEntitiesWithLimit() throws Exception { - Set result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - 2L, null, null, null, null, null, null, null, null, null, null, - null); + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(2L, null, null, null, null, null, null, + null, null), new TimelineDataToRetrieve()); Assert.assertEquals(2, result.size()); // Needs to be rewritten once hashcode and equals for // TimelineEntity is implemented @@ -357,10 +366,11 @@ public void testGetEntitiesWithLimit() throws Exception { Assert.fail("Entity not sorted by created time"); } } - result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - 3L, null, null, null, null, null, null, null, null, null, null, - null); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", 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 // is left out due to limit Assert.assertEquals(3, result.size()); @@ -369,10 +379,12 @@ public void testGetEntitiesWithLimit() throws Exception { @Test public void testGetEntitiesByTimeWindows() throws Exception { // Get entities based on created time start and end time range. - Set result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, 1425016502030L, 1425016502060L, null, null, null, null, null, - null, null, null, null); + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, 1425016502030L, 1425016502060L, null, + null, null, null, null, null), + new TimelineDataToRetrieve()); Assert.assertEquals(1, result.size()); // Only one entity with ID id_4 should be returned. for (TimelineEntity entity : result) { @@ -382,10 +394,12 @@ public void testGetEntitiesByTimeWindows() throws Exception { } // Get entities if only created time end is specified. - result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, null, 1425016502010L, null, null, null, null, null, null, - null, null, null); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, null, 1425016502010L, null, null, + null, null, null, null), + new TimelineDataToRetrieve()); Assert.assertEquals(3, result.size()); for (TimelineEntity entity : result) { if (entity.getId().equals("id_4")) { @@ -394,10 +408,12 @@ public void testGetEntitiesByTimeWindows() throws Exception { } // Get entities if only created time start is specified. - result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, 1425016502010L, null, null, null, null, null, null, null, - null, null, null); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, 1425016502010L, null, null, null, + null, null, null, null), + new TimelineDataToRetrieve()); Assert.assertEquals(1, result.size()); for (TimelineEntity entity : result) { if (!entity.getId().equals("id_4")) { @@ -411,10 +427,12 @@ public void testGetFilteredEntities() throws Exception { // Get entities based on info filters. Map infoFilters = new HashMap(); infoFilters.put("info2", 3.5); - Set result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, null, null, null, null, infoFilters, null, null, null, null, - null, null); + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, null, null, null, null, infoFilters, + null, null, null), + new TimelineDataToRetrieve()); Assert.assertEquals(1, result.size()); // Only one entity with ID id_3 should be returned. for (TimelineEntity entity : result) { @@ -427,10 +445,12 @@ public void testGetFilteredEntities() throws Exception { Map configFilters = new HashMap(); configFilters.put("config_1", "123"); configFilters.put("config_3", "abc"); - result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, null, null, null, null, null, configFilters, null, null, null, - null, null); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, null, null, null, null, null, + configFilters, null, null), + new TimelineDataToRetrieve()); Assert.assertEquals(2, result.size()); for (TimelineEntity entity : result) { if (!entity.getId().equals("id_1") && !entity.getId().equals("id_3")) { @@ -442,10 +462,12 @@ public void testGetFilteredEntities() throws Exception { Set eventFilters = new HashSet(); eventFilters.add("event_2"); eventFilters.add("event_4"); - result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, null, null, null, null, null, null, null, eventFilters, null, - null, null); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, null, null, null, null, null, null, + null, eventFilters), + new TimelineDataToRetrieve()); Assert.assertEquals(1, result.size()); for (TimelineEntity entity : result) { if (!entity.getId().equals("id_3")) { @@ -456,10 +478,12 @@ public void testGetFilteredEntities() throws Exception { // Get entities based on metric filters. Set metricFilters = new HashSet(); metricFilters.add("metric3"); - result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, null, null, null, null, null, null, metricFilters, null, null, - null, null); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, null, null, null, null, null, null, + metricFilters, null), + new TimelineDataToRetrieve()); Assert.assertEquals(2, result.size()); // Two entities with IDs' id_1 and id_2 should be returned. for (TimelineEntity entity : result) { @@ -476,10 +500,12 @@ public void testGetEntitiesByRelations() throws Exception { Set relatesToIds = new HashSet(); relatesToIds.add("flow1"); relatesTo.put("flow", relatesToIds); - Set result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, null, null, relatesTo, null, null, null, null, null, null, - null, null); + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, null, null, relatesTo, null, null, + null, null, null), + new TimelineDataToRetrieve()); Assert.assertEquals(1, result.size()); // Only one entity with ID id_1 should be returned. for (TimelineEntity entity : result) { @@ -493,10 +519,12 @@ public void testGetEntitiesByRelations() throws Exception { Set isRelatedToIds = new HashSet(); isRelatedToIds.add("tid1_2"); isRelatedTo.put("type1", isRelatedToIds); - result = - reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app", - null, null, null, null, isRelatedTo, null, null, null, null, null, - null, null); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters(null, null, null, null, isRelatedTo, null, + null, null, null), + new TimelineDataToRetrieve()); Assert.assertEquals(2, result.size()); // Two entities with IDs' id_1 and id_3 should be returned. for (TimelineEntity entity : result) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java index b74f5a1d516..4e07ecffe1a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java @@ -49,6 +49,9 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type; 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.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; @@ -574,9 +577,11 @@ public void testWriteApplicationToHBase() throws Exception { matchMetrics(metricValues, metricMap); // read the timeline entity using the reader this time - TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appId, - entity.getType(), entity.getId(), null, null, - EnumSet.of(TimelineReader.Field.ALL)); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appId, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve( + null, null, EnumSet.of(TimelineReader.Field.ALL))); assertNotNull(e1); // verify attributes @@ -771,12 +776,15 @@ public void testWriteEntityToHBase() throws Exception { assertEquals(16, colCount); // read the timeline entity using the reader this time - TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName, - entity.getType(), entity.getId(), null, null, - EnumSet.of(TimelineReader.Field.ALL)); - Set es1 = reader.getEntities(user, cluster, flow, runid, - appName, entity.getType(), null, null, null, null, null, null, null, - null, null, null, null, EnumSet.of(TimelineReader.Field.ALL)); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); + Set es1 = reader.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); assertNotNull(e1); assertEquals(1, es1.size()); @@ -906,12 +914,14 @@ public void testEvents() throws IOException { } // read the timeline entity using the reader this time - TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName, - entity.getType(), entity.getId(), null, null, - EnumSet.of(TimelineReader.Field.ALL)); - TimelineEntity e2 = reader.getEntity(user, cluster, null, null, appName, - entity.getType(), entity.getId(), null, null, - EnumSet.of(TimelineReader.Field.ALL)); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); + TimelineEntity e2 = reader.getEntity( + new TimelineReaderContext(cluster, user, null, null, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); assertNotNull(e1); assertNotNull(e2); assertEquals(e1, e2); @@ -1012,12 +1022,15 @@ public void testEventsWithEmptyInfo() throws IOException { assertEquals(1, rowCount); // read the timeline entity using the reader this time - TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName, - entity.getType(), entity.getId(), null, null, - EnumSet.of(TimelineReader.Field.ALL)); - Set es1 = reader.getEntities(user, cluster, flow, runid, - appName, entity.getType(), null, null, null, null, null, null, null, - null, null, null, null, EnumSet.of(TimelineReader.Field.ALL)); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); + Set es1 = reader.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); assertNotNull(e1); assertEquals(1, es1.size()); @@ -1100,31 +1113,36 @@ public void testNonIntegralMetricValues() throws IOException { @Test public void testReadEntities() throws Exception { - TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name", - 1002345678919L, "application_1231111111_1111","world", "hello", null, - null, EnumSet.of(Field.ALL)); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", "hello"), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); assertNotNull(e1); assertEquals(3, e1.getConfigs().size()); assertEquals(1, e1.getIsRelatedToEntities().size()); - Set es1 = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, "application_1231111111_1111","world", - null, null, null, null, null, null, null, null, null, null, null, - EnumSet.of(Field.ALL)); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", + null), new TimelineEntityFilters(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); assertEquals(3, es1.size()); } @Test public void testReadEntitiesDefaultView() throws Exception { - TimelineEntity e1 = - reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L, - "application_1231111111_1111","world", "hello", null, null, null); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", "hello"), + new TimelineDataToRetrieve()); assertNotNull(e1); assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() && e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() && e1.getRelatesToEntities().isEmpty()); - Set es1 = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, "application_1231111111_1111","world", - null, null, null, null, null, null, null, null, null, null, null, null); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve()); assertEquals(3, es1.size()); for (TimelineEntity e : es1) { assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() && @@ -1135,17 +1153,20 @@ public void testReadEntitiesDefaultView() throws Exception { @Test public void testReadEntitiesByFields() throws Exception { - TimelineEntity e1 = - reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L, - "application_1231111111_1111","world", "hello", null, null, - EnumSet.of(Field.INFO, Field.CONFIGS)); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", "hello"), + new TimelineDataToRetrieve( + null, null, EnumSet.of(Field.INFO, Field.CONFIGS))); assertNotNull(e1); assertEquals(3, e1.getConfigs().size()); assertEquals(0, e1.getIsRelatedToEntities().size()); - Set es1 = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, "application_1231111111_1111","world", - null, null, null, null, null, null, null, null, null, null, null, - EnumSet.of(Field.IS_RELATED_TO, Field.METRICS)); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve( + null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS))); assertEquals(3, es1.size()); int metricsCnt = 0; int isRelatedToCnt = 0; @@ -1165,14 +1186,17 @@ public void testReadEntitiesConfigPrefix() throws Exception { TimelineFilterList list = new TimelineFilterList(Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); - TimelineEntity e1 = - reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L, - "application_1231111111_1111","world", "hello", list, null, null); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", "hello"), + new TimelineDataToRetrieve(list, null, null)); assertNotNull(e1); assertEquals(1, e1.getConfigs().size()); - Set es1 = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, "application_1231111111_1111","world", - null, null, null, null, null, null, null, null, null, list, null, null); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve(list, null, null)); int cfgCnt = 0; for (TimelineEntity entity : es1) { cfgCnt += entity.getConfigs().size(); @@ -1186,10 +1210,12 @@ public void testReadEntitiesConfigFilterPrefix() throws Exception { TimelineFilterList list = new TimelineFilterList(Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); - Set entities = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, "application_1231111111_1111","world", - null, null, null, null, null, null, confFilters, null, null, list, null, - null); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", null), + new TimelineEntityFilters(null, null, null, null, null, null, + confFilters, null, null), + new TimelineDataToRetrieve(list, null, null)); assertEquals(1, entities.size()); int cfgCnt = 0; for (TimelineEntity entity : entities) { @@ -1203,14 +1229,17 @@ public void testReadEntitiesMetricPrefix() throws Exception { TimelineFilterList list = new TimelineFilterList(Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); - TimelineEntity e1 = - reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L, - "application_1231111111_1111","world", "hello", null, list, null); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", "hello"), + new TimelineDataToRetrieve(null, list, null)); assertNotNull(e1); assertEquals(1, e1.getMetrics().size()); - Set es1 = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, "application_1231111111_1111","world", - null, null, null, null, null, null, null, null, null, null, list, null); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve(null, list, null)); int metricCnt = 0; for (TimelineEntity entity : es1) { metricCnt += entity.getMetrics().size(); @@ -1224,10 +1253,12 @@ public void testReadEntitiesMetricFilterPrefix() throws Exception { TimelineFilterList list = new TimelineFilterList(Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); - Set entities = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, "application_1231111111_1111","world", - null, null, null, null, null, null, null, metricFilters, null, null, - list, null); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1","user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111","world", null), + new TimelineEntityFilters(null, null, null, null, null, null, null, + metricFilters, null), + new TimelineDataToRetrieve(null, list, null)); assertEquals(1, entities.size()); int metricCnt = 0; for (TimelineEntity entity : entities) { @@ -1238,33 +1269,40 @@ public void testReadEntitiesMetricFilterPrefix() throws Exception { @Test 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", - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, - EnumSet.of(Field.ALL)); + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); assertNotNull(e1); assertEquals(3, e1.getConfigs().size()); assertEquals(1, e1.getIsRelatedToEntities().size()); - Set es1 = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, null, - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, - null, null, null, null, null, null, null, EnumSet.of(Field.ALL)); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL))); assertEquals(3, es1.size()); } @Test 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", - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null); + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve()); assertNotNull(e1); assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() && e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() && e1.getRelatesToEntities().isEmpty()); - Set es1 = reader.getEntities("user1", "cluster1", - "some_flow_name", 1002345678919L, null, - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, - null, null, null, null, null, null, null, null); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve()); assertEquals(3, es1.size()); for (TimelineEntity e : es1) { assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() && @@ -1275,18 +1313,22 @@ public void testReadAppsDefaultView() throws Exception { @Test 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", - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, - EnumSet.of(Field.INFO, Field.CONFIGS)); + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve( + null, null, EnumSet.of(Field.INFO, Field.CONFIGS))); assertNotNull(e1); assertEquals(3, e1.getConfigs().size()); assertEquals(0, e1.getIsRelatedToEntities().size()); Set es1 = reader.getEntities( - "user1", "cluster1", "some_flow_name", 1002345678919L, null, - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, - null, null, null, null, null, null, null, - EnumSet.of(Field.IS_RELATED_TO, Field.METRICS)); + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve( + null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS))); assertEquals(3, es1.size()); int metricsCnt = 0; int isRelatedToCnt = 0; @@ -1306,15 +1348,19 @@ public void testReadAppsConfigPrefix() throws Exception { TimelineFilterList list = new TimelineFilterList(Operator.OR, 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", - TimelineEntityType.YARN_APPLICATION.toString(), null, list, null, null); + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve(list, null, null)); assertNotNull(e1); assertEquals(1, e1.getConfigs().size()); Set es1 = reader.getEntities( - "user1", "cluster1", "some_flow_name", 1002345678919L, null, - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, - null, null, null, null, null, list, null, null); + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null) , + new TimelineEntityFilters(), + new TimelineDataToRetrieve(list, null, null)); int cfgCnt = 0; for (TimelineEntity entity : es1) { cfgCnt += entity.getConfigs().size(); @@ -1329,9 +1375,12 @@ public void testReadAppsConfigFilterPrefix() throws Exception { new TimelineFilterList(Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); Set entities = reader.getEntities( - "user1", "cluster1", "some_flow_name", 1002345678919L, null, - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, - null, null, confFilters, null, null, list, null, null); + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters(null, null, null, null, null, null, + confFilters, null, null), + new TimelineDataToRetrieve(list, null, null)); assertEquals(1, entities.size()); int cfgCnt = 0; for (TimelineEntity entity : entities) { @@ -1345,15 +1394,19 @@ public void testReadAppsMetricPrefix() throws Exception { TimelineFilterList list = new TimelineFilterList(Operator.OR, 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", - TimelineEntityType.YARN_APPLICATION.toString(), null, null, list, null); + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve(null, list, null)); assertNotNull(e1); assertEquals(1, e1.getMetrics().size()); Set es1 = reader.getEntities( - "user1", "cluster1", "some_flow_name", 1002345678919L, null, - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, - null, null, null, null, null, null, list, null); + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve(null, list, null)); int metricCnt = 0; for (TimelineEntity entity : es1) { metricCnt += entity.getMetrics().size(); @@ -1368,9 +1421,12 @@ public void testReadAppsMetricFilterPrefix() throws Exception { new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); Set metricFilters = ImmutableSet.of("MAP1_SLOT_MILLIS"); Set entities = reader.getEntities( - "user1", "cluster1", "some_flow_name", 1002345678919L, null, - TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null, - null, null, null, metricFilters, null, null, list, null); + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters(null, null, null, null, null, null, null, + metricFilters, null), + new TimelineDataToRetrieve(null, list, null)); int metricCnt = 0; assertEquals(1, entities.size()); for (TimelineEntity entity : entities) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java index cb3f7eefa94..9161902c7a9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java @@ -45,6 +45,9 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; 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.HBaseTimelineWriterImpl; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator; @@ -179,10 +182,12 @@ public void testWriteFlowRunMinMax() throws Exception { hbr.init(c1); hbr.start(); // get the flow activity entity - Set entities = - hbr.getEntities(null, cluster, null, null, null, - TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null, - null, null, null, null, null, null, null, null, null); + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, null, null, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null), + new TimelineEntityFilters(10L, null, null, null, null, null, + null, null, null), + new TimelineDataToRetrieve()); assertEquals(1, entities.size()); for (TimelineEntity e : entities) { FlowActivityEntity flowActivity = (FlowActivityEntity)e; @@ -235,10 +240,12 @@ public void testWriteFlowActivityOneFlow() throws Exception { hbr.init(c1); hbr.start(); - Set entities = - hbr.getEntities(user, cluster, flow, null, null, - TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null, - null, null, null, null, null, null, null, null, null); + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null), + new TimelineEntityFilters(10L, null, null, null, null, null, + null, null, null), + new TimelineDataToRetrieve()); assertEquals(1, entities.size()); for (TimelineEntity e : entities) { FlowActivityEntity entity = (FlowActivityEntity)e; @@ -350,10 +357,12 @@ public void testFlowActivityTableOneFlowMultipleRunIds() throws IOException { hbr.init(c1); hbr.start(); - Set entities = - hbr.getEntities(null, cluster, null, null, null, - TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null, - null, null, null, null, null, null, null, null, null); + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, null, null, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null), + new TimelineEntityFilters(10L, null, null, null, null, null, + null, null, null), + new TimelineDataToRetrieve()); assertEquals(1, entities.size()); for (TimelineEntity e : entities) { FlowActivityEntity flowActivity = (FlowActivityEntity)e; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java index 57c326b3dcd..95047992108 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java @@ -44,7 +44,9 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; -import org.apache.hadoop.yarn.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.TimelineFilterList; import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; @@ -183,8 +185,10 @@ public void testWriteFlowRunMinMax() throws Exception { hbr.init(c1); hbr.start(); // get the flow run entity - TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null, - TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, null); + TimelineEntity entity = hbr.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineDataToRetrieve()); assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); FlowRunEntity flowRun = (FlowRunEntity)entity; assertEquals(minStartTs, flowRun.getStartTime()); @@ -242,8 +246,10 @@ public void testWriteFlowRunMetricsOneFlow() throws Exception { hbr = new HBaseTimelineReaderImpl(); hbr.init(c1); hbr.start(); - TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null, - TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, null); + TimelineEntity entity = hbr.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineDataToRetrieve()); assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); Set metrics = entity.getMetrics(); assertEquals(2, metrics.size()); @@ -350,9 +356,10 @@ public void testWriteFlowRunMetricsPrefix() throws Exception { TimelineFilterList metricsToRetrieve = new TimelineFilterList(new TimelinePrefixFilter(TimelineCompareOp.EQUAL, metric1.substring(0, metric1.indexOf("_") + 1))); - TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null, - TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, - metricsToRetrieve, null); + TimelineEntity entity = hbr.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineDataToRetrieve(null, metricsToRetrieve, null)); assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); Set metrics = entity.getMetrics(); assertEquals(1, metrics.size()); @@ -373,9 +380,11 @@ public void testWriteFlowRunMetricsPrefix() throws Exception { } } - Set entities = hbr.getEntities(user, cluster, flow, runid, - null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, - null, null, null, null, null, null, null, metricsToRetrieve, null); + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve(null, metricsToRetrieve, null)); assertEquals(1, entities.size()); for (TimelineEntity timelineEntity : entities) { Set timelineMetrics = timelineEntity.getMetrics(); @@ -441,18 +450,21 @@ public void testWriteFlowRunsMetricFields() throws Exception { hbr = new HBaseTimelineReaderImpl(); hbr.init(c1); hbr.start(); - Set entities = hbr.getEntities(user, cluster, flow, runid, - null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, - null, null, null, null, null, null, null, null, null); + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve()); assertEquals(1, entities.size()); for (TimelineEntity timelineEntity : entities) { assertEquals(0, timelineEntity.getMetrics().size()); } - entities = hbr.getEntities(user, cluster, flow, runid, - null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, - null, null, null, null, null, null, null, null, - EnumSet.of(Field.METRICS)); + entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS))); assertEquals(1, entities.size()); for (TimelineEntity timelineEntity : entities) { Set timelineMetrics = timelineEntity.getMetrics();