diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java index 0701001..66bf191 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java @@ -523,6 +523,19 @@ public void setModifiedTime(long modifiedTime) { } } + /** + * Set UID in info which will be then used for query by UI. + * @param uidKey key for UID in info. + * @param uId UID to be set for the key. + */ + public void setUID(String uidKey, String uId) { + if (real == null) { + info.put(uidKey, uId); + } else { + real.addInfo(uidKey, uId); + } + } + public boolean isValid() { return (getId() != null && getType() != null); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/TimelineContext.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/TimelineContext.java new file mode 100644 index 0000000..694b709 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/TimelineContext.java @@ -0,0 +1,146 @@ +/** + * 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; + +/** + * Encapsulates timeline context information. + */ +public class TimelineContext { + + private String clusterId; + private String userId; + private String flowName; + private Long flowRunId; + private String appId; + + public TimelineContext() { + this(null, null, null, 0L, null); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((appId == null) ? 0 : appId.hashCode()); + result = prime * result + ((clusterId == null) ? 0 : clusterId.hashCode()); + result = prime * result + ((flowName == null) ? 0 : flowName.hashCode()); + result = prime * result + ((flowRunId == null) ? 0 : flowRunId.hashCode()); + result = prime * result + ((userId == null) ? 0 : userId.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelineContext other = (TimelineContext) obj; + if (appId == null) { + if (other.appId != null) { + return false; + } + } else if (!appId.equals(other.appId)) { + return false; + } + if (clusterId == null) { + if (other.clusterId != null) { + return false; + } + } else if (!clusterId.equals(other.clusterId)) { + return false; + } + if (flowName == null) { + if (other.flowName != null) { + return false; + } + } else if (!flowName.equals(other.flowName)) { + return false; + } + if (flowRunId == null) { + if (other.flowRunId != null) { + return false; + } + } else if (!flowRunId.equals(other.flowRunId)) { + return false; + } + if (userId == null) { + if (other.userId != null) { + return false; + } + } else if (!userId.equals(other.userId)) { + return false; + } + return true; + } + + public TimelineContext(String clusterId, String userId, String flowName, + Long flowRunId, String appId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + this.appId = appId; + } + + public String getClusterId() { + return clusterId; + } + + public void setClusterId(String cluster) { + this.clusterId = cluster; + } + + public String getUserId() { + return userId; + } + + public void setUserId(String user) { + this.userId = user; + } + + public String getFlowName() { + return flowName; + } + + public void setFlowName(String flow) { + this.flowName = flow; + } + + public Long getFlowRunId() { + return flowRunId; + } + + public void setFlowRunId(long runId) { + this.flowRunId = runId; + } + + public String getAppId() { + return appId; + } + + public void setAppId(String app) { + this.appId = app; + } +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java index 6cc477f..03a8ea6 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java @@ -18,74 +18,61 @@ package org.apache.hadoop.yarn.server.timelineservice.collector; -public class TimelineCollectorContext { +import org.apache.hadoop.yarn.server.timelineservice.TimelineContext; + +/** + * Encapsulates context information required by collector during a put. + */ +public class TimelineCollectorContext extends TimelineContext { - private String clusterId; - private String userId; - private String flowName; private String flowVersion; - private long flowRunId; - private String appId; public TimelineCollectorContext() { this(null, null, null, null, 0L, null); } public TimelineCollectorContext(String clusterId, String userId, - String flowName, String flowVersion, long flowRunId, String appId) { - this.clusterId = clusterId; - this.userId = userId; - this.flowName = flowName; + String flowName, String flowVersion, Long flowRunId, String appId) { + super(clusterId, userId, flowName, flowRunId, appId); this.flowVersion = flowVersion; - this.flowRunId = flowRunId; - this.appId = appId; - } - - public String getClusterId() { - return clusterId; - } - - public void setClusterId(String clusterId) { - this.clusterId = clusterId; - } - - public String getUserId() { - return userId; - } - - public void setUserId(String userId) { - this.userId = userId; } - public String getFlowName() { - return flowName; + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = + prime * result + ((flowVersion == null) ? 0 : flowVersion.hashCode()); + return result + super.hashCode(); } - public void setFlowName(String flowName) { - this.flowName = flowName; + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelineCollectorContext other = (TimelineCollectorContext) obj; + if (flowVersion == null) { + if (other.flowVersion != null) { + return false; + } + } else if (!flowVersion.equals(other.flowVersion)) { + return false; + } + return super.equals(obj); } public String getFlowVersion() { return flowVersion; } - public void setFlowVersion(String flowVersion) { - this.flowVersion = flowVersion; - } - - public long getFlowRunId() { - return flowRunId; - } - - public void setFlowRunId(long flowRunId) { - this.flowRunId = flowRunId; - } - - public String getAppId() { - return appId; - } - - public void setAppId(String appId) { - this.appId = appId; + public void setFlowVersion(String version) { + this.flowVersion = version; } -} +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/package-info.java new file mode 100644 index 0000000..58e23f0 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.server.timelineservice contains classes to be used + * across timeline reader and collector. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java new file mode 100644 index 0000000..1aa2d4b --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java @@ -0,0 +1,92 @@ +/** + * 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 org.apache.hadoop.yarn.server.timelineservice.TimelineContext; + +/** + * Encapsulates fields necessary to make a query in timeline reader. + */ +public class TimelineReaderContext extends TimelineContext { + + private String entityType; + private String entityId; + public TimelineReaderContext(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, String entityId) { + super(clusterId, userId, flowName, flowRunId, appId); + this.entityType = entityType; + this.entityId = entityId; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + ((entityId == null) ? 0 : entityId.hashCode()); + result = + prime * result + ((entityType == null) ? 0 : entityType.hashCode()); + result = result + super.hashCode(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelineReaderContext other = (TimelineReaderContext) obj; + if (entityId == null) { + if (other.entityId != null) { + return false; + } + } else if (!entityId.equals(other.entityId)) { + return false; + } + if (entityType == null) { + if (other.entityType != null) { + return false; + } + } else if (!entityType.equals(other.entityType)) { + return false; + } + return super.equals(obj); + } + + public String getEntityType() { + return entityType; + } + + public void setEntityType(String type) { + this.entityType = type; + } + + public String getEntityId() { + return entityId; + } + + public void setEntityId(String id) { + this.entityId = id; + } +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java 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 37f1671..b63129d 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java @@ -27,15 +27,22 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.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; + @Private @Unstable public class TimelineReaderManager extends AbstractService { + @VisibleForTesting + static final String UID_KEY = "UID"; private TimelineReader reader; public TimelineReaderManager(TimelineReader timelineReader) { @@ -59,13 +66,63 @@ private static String getClusterID(String clusterId, Configuration conf) { return clusterId; } + private static TimelineEntityType getTimelineEntityType(String entityType) { + if (entityType == null) { + return null; + } + try { + return TimelineEntityType.valueOf(entityType); + } catch (IllegalArgumentException e) { + return null; + } + } + + /** + * Fill UID in the info field of entity based on the query(identified by + * entity type). + * @param entityType Entity type of query. + * @param entity Timeline Entity. + * @param context Context defining the query. + */ + private static void fillUID(TimelineEntityType entityType, + TimelineEntity entity, TimelineReaderContext context) { + if (entityType != null) { + switch(entityType) { + case YARN_FLOW_ACTIVITY: + FlowActivityEntity activityEntity = (FlowActivityEntity)entity; + context.setUserId(activityEntity.getUser()); + context.setFlowName(activityEntity.getFlowName()); + entity.setUID(UID_KEY, + TimelineUIDConverter.FLOW_UID.encodeUID(context)); + return; + case YARN_FLOW_RUN: + FlowRunEntity runEntity = (FlowRunEntity)entity; + context.setFlowRunId(runEntity.getRunId()); + entity.setUID(UID_KEY, + TimelineUIDConverter.FLOWRUN_UID.encodeUID(context)); + return; + case YARN_APPLICATION: + context.setAppId(entity.getId()); + entity.setUID(UID_KEY, + TimelineUIDConverter.APPLICATION_UID.encodeUID(context)); + return; + default: + break; + } + } + context.setEntityType(entity.getType()); + context.setEntityId(entity.getId()); + entity.setUID(UID_KEY, + TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context)); + } + /** * Get a set of entities matching given predicates. The meaning of each * argument has been documented with {@link TimelineReader#getEntities}. * * @see TimelineReader#getEntities */ - Set getEntities(String userId, String clusterId, + public Set getEntities(String userId, String clusterId, String flowName, Long flowRunId, String appId, String entityType, Long limit, Long createdTimeBegin, Long createdTimeEnd, Long modifiedTimeBegin, Long modifiedTimeEnd, @@ -74,10 +131,20 @@ private static String getClusterID(String clusterId, Configuration conf) { Set metricFilters, Set eventFilters, EnumSet fieldsToRetrieve) throws IOException { String cluster = getClusterID(clusterId, getConfig()); - return reader.getEntities(userId, cluster, flowName, flowRunId, appId, + Set entities = + reader.getEntities(userId, cluster, flowName, flowRunId, appId, entityType, limit, createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters, eventFilters, null, null, fieldsToRetrieve); + if (entities != null) { + TimelineEntityType type = getTimelineEntityType(entityType); + TimelineReaderContext context = new TimelineReaderContext(cluster, userId, + flowName, flowRunId, appId, entityType, null); + for (TimelineEntity entity : entities) { + fillUID(type, entity, context); + } + } + return entities; } /** @@ -90,7 +157,16 @@ 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()); - return reader.getEntity(userId, cluster, flowName, flowRunId, appId, + TimelineEntity entity = + reader.getEntity(userId, cluster, flowName, flowRunId, appId, entityType, entityId, null, null, fields); + + if (entity != null) { + TimelineEntityType type = getTimelineEntityType(entityType); + TimelineReaderContext context = new TimelineReaderContext(cluster, userId, + flowName, flowRunId, appId, entityType, null); + fillUID(type, entity, context); + } + return entity; } } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java new file mode 100644 index 0000000..8f4f957 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java @@ -0,0 +1,364 @@ +/** + * 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.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import javax.servlet.http.HttpServletRequest; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; + +/** + * Set of utility methods to be used across timeline reader. + */ +public final class TimelineReaderUtils { + private TimelineReaderUtils() { + } + + /** + * Parse a delimited string and convert it into a set of strings. For + * instance, if delimiter is ",", then the string should be represented as + * "value1,value2,value3". + * @param str delimited string. + * @param delimiter string is delimited by this delimiter. + * @return set of strings. + */ + public static Set parseValuesStr(String str, String delimiter) { + if (str == null || str.isEmpty()) { + return null; + } + Set strSet = new HashSet(); + String[] strs = str.split(delimiter); + for (String aStr : strs) { + strSet.add(aStr.trim()); + } + return strSet; + } + + @SuppressWarnings("unchecked") + private static void parseKeyValues(Map map, String str, + String pairsDelim, String keyValuesDelim, boolean stringValue, + boolean multipleValues) { + String[] pairs = str.split(pairsDelim); + for (String pair : pairs) { + if (pair == null || pair.trim().isEmpty()) { + continue; + } + String[] pairStrs = pair.split(keyValuesDelim); + if (pairStrs.length < 2) { + continue; + } + if (!stringValue) { + try { + Object value = + GenericObjectMapper.OBJECT_READER.readValue(pairStrs[1].trim()); + map.put(pairStrs[0].trim(), (T) value); + } catch (IOException e) { + map.put(pairStrs[0].trim(), (T) pairStrs[1].trim()); + } + } else { + String key = pairStrs[0].trim(); + if (multipleValues) { + Set values = new HashSet(); + for (int i = 1; i < pairStrs.length; i++) { + values.add(pairStrs[i].trim()); + } + map.put(key, (T) values); + } else { + map.put(key, (T) pairStrs[1].trim()); + } + } + } + } + + /** + * Parse a delimited string and convert it into a map of key-values with each + * key having a set of values. Both the key and values are interpreted as + * strings. + * For instance, if pairsDelim is "," and keyValuesDelim is ":", then the + * string should be represented as + * "key1:value11:value12:value13,key2:value21,key3:value31:value32". + * @param str delimited string represented as multiple keys having multiple + * values. + * @param pairsDelim key-values pairs are delimited by this delimiter. + * @param keyValuesDelim values for a key are delimited by this delimiter. + * @return a map of key-values with each key having a set of values. + */ + public static Map> parseKeyStrValuesStr(String str, + String pairsDelim, String keyValuesDelim) { + if (str == null) { + return null; + } + Map> map = new HashMap>(); + parseKeyValues(map, str, pairsDelim, keyValuesDelim, true, true); + return map; + } + + /** + * Parse a delimited string and convert it into a map of key-value pairs with + * both the key and value interpreted as strings. + * For instance, if pairsDelim is "," and keyValDelim is ":", then the string + * should be represented as "key1:value1,key2:value2,key3:value3". + * @param str delimited string represented as key-value pairs. + * @param pairsDelim key-value pairs are delimited by this delimiter. + * @param keyValuesDelim key and value are delimited by this delimiter. + * @return a map of key-value pairs with both key and value being strings. + */ + public static Map parseKeyStrValueStr(String str, + String pairsDelim, String keyValDelim) { + if (str == null) { + return null; + } + Map map = new HashMap(); + parseKeyValues(map, str, pairsDelim, keyValDelim, true, false); + return map; + } + + /** + * Parse a delimited string and convert it into a map of key-value pairs with + * key being a string and value interpreted as any object. + * For instance, if pairsDelim is "," and keyValDelim is ":", then the string + * should be represented as "key1:value1,key2:value2,key3:value3". + * @param str delimited string represented as key-value pairs. + * @param pairsDelim key-value pairs are delimited by this delimiter. + * @param keyValuesDelim key and value are delimited by this delimiter. + * @return a map of key-value pairs with key being a string and value amy + * object. + */ + public static Map parseKeyStrValueObj(String str, + String pairsDelim, String keyValDelim) { + if (str == null) { + return null; + } + Map map = new HashMap(); + parseKeyValues(map, str, pairsDelim, keyValDelim, false, false); + return map; + } + + /** + * Interprets passed string as set of fields delimited by passed delimiter. + * For instance, if delimiter is ",", then the passed string should be + * represented as "METRICS,CONFIGS" where the delimited parts of the string + * present in {@link Field}. + * @param str passed string. + * @param delimiter string delimiter. + * @return a set of {@link Field}. + */ + public static EnumSet parseFieldsStr(String str, String delimiter) { + if (str == null) { + return null; + } + String[] strs = str.split(delimiter); + EnumSet fieldList = EnumSet.noneOf(Field.class); + for (String s : strs) { + fieldList.add(Field.valueOf(s.trim().toUpperCase())); + } + return fieldList; + } + + /** + * Interpret passed string as a long. + * @param str Passed string. + * @return long representation if string is not null, null otherwise. + */ + public static Long parseLongStr(String str) { + return str == null ? null : Long.parseLong(str.trim()); + } + + /** + * Trims the passed string is its not null. + * @param str Passed string. + * @return trimmed string if string is not null, null otherwise. + */ + public static String parseStr(String str) { + return str == null ? null : str.trim(); + } + + /** + * Get UGI from HTTP request. + * @param req HTTP request. + * @return UGI. + */ + public static UserGroupInformation getUser(HttpServletRequest req) { + String remoteUser = req.getRemoteUser(); + UserGroupInformation callerUGI = null; + if (remoteUser != null) { + callerUGI = UserGroupInformation.createRemoteUser(remoteUser); + } + return callerUGI; + } + + /** + * Split the passed string along the passed delimiter character while looking + * for escape char to interpret the splitted parts correctly. For delimiter or + * escape character to be interpreted as part of the string, they have to be + * escaped by putting an escape character in front. + * @param str string to be split. + * @param delimiterChar delimiter used for splitting. + * @param escapeChar delimiter and escape character will be escaped using this + * character. + * @return a list of strings after split. + * @throws IllegalArgumentException if string is not properly escaped. + */ + public static List split(final String str, final char delimiterChar, + final char escapeChar) throws IllegalArgumentException { + if (str == null) { + return null; + } + int len = str.length(); + if (len == 0) { + return Collections.emptyList(); + } + List list = new ArrayList(); + // Keeps track of offset of the passed string. + int offset = 0; + // Indicates start offset from which characters will be copied from original + // string to destination string. Resets when an escape or delimiter char is + // encountered. + int startOffset = 0; + StringBuilder builder = new StringBuilder(len); + // Iterate over the string till we reach the end. + while (offset < len) { + if (str.charAt(offset) == escapeChar) { + // An escape character must be followed by a delimiter or escape char + // but we have reached the end and have no further character to look at. + if (offset + 1 >= len) { + throw new IllegalArgumentException( + "Escape char not properly escaped."); + } + char nextChar = str.charAt(offset + 1); + // Next character must be a delimiter or an escape char. + if (nextChar != escapeChar && nextChar != delimiterChar) { + throw new IllegalArgumentException( + "Escape char or delimiter char not properly escaped."); + } + // Copy contents from the offset where last escape or delimiter char was + // encountered. + if (startOffset < offset) { + builder.append(str.substring(startOffset, offset)); + } + builder.append(nextChar); + offset += 2; + // Reset the start offset as an escape char has been encountered. + startOffset = offset; + continue; + } else if (str.charAt(offset) == delimiterChar) { + // A delimiter has been encountered without an escape character. + // 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()); + // Reset the start offset as a delimiter has been encountered. + startOffset = ++offset; + builder = new StringBuilder(len - offset); + continue; + } + offset++; + } + // Copy rest of the characters. + if (!str.isEmpty()) { + builder.append(str.substring(startOffset)); + } + // Add the last part of delimited string to list. + list.add(builder.toString()); + return list; + } + + private static String escapeString(final String str, final char delimiterChar, + final char escapeChar) { + if (str == null) { + return null; + } + int len = str.length(); + if (len == 0) { + return ""; + } + StringBuilder builder = new StringBuilder(); + // Keeps track of offset of the passed string. + int offset = 0; + // Indicates start offset from which characters will be copied from original + // string to destination string. Resets when an escape or delimiter char is + // encountered. + int startOffset = 0; + // Iterate over the string till we reach the end. + while (offset < len) { + char charAtOffset = str.charAt(offset); + if (charAtOffset == escapeChar || charAtOffset == delimiterChar) { + // If an escape or delimiter character is encountered, copy characters + // from the offset where escape or delimiter was last encountered. + if (startOffset < offset) { + builder.append(str.substring(startOffset, offset)); + } + // Append escape char before delimiter/escape char. + builder.append(escapeChar).append(charAtOffset); + // Reset start offset for copying characters when next escape/delimiter + // char is encountered. + startOffset = offset + 1; + } + offset++; + } + // Copy remaining characters. + builder.append(str.substring(startOffset)); + return builder.toString(); + } + + /** + * Join different strings in the passed string array delimited by passed + * delimiter with delimiter and escape character escaped using passed escape + * char. + * @param strs strings to be joined. + * @param delimiterChar delimiter used to join strings. + * @param escapeChar escape character used to escape delimiter and escape + * char. + * @return a single string joined using delimiter and properly escaped. + */ + public static String joinAndEscapeStrings(final String[] strs, + final char delimiterChar, final char escapeChar) { + int len = strs.length; + // Escape each string in string array. + for (int index = 0; index < len; index++) { + if (strs[index] == null) { + return null; + } + strs[index] = escapeString(strs[index], delimiterChar, escapeChar); + } + // Join the strings after they have been escaped. + return StringUtils.join(strs, delimiterChar); + } + + /** + * Get username from caller UGI. + * @param callerUGI caller UGI. + * @return username. + */ + public static String getUserName(UserGroupInformation callerUGI) { + return ((callerUGI != null) ? callerUGI.getUserName().trim() : ""); + } +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java 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 f0283a5..3688ab0 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java @@ -18,17 +18,12 @@ package org.apache.hadoop.yarn.server.timelineservice.reader; -import java.io.IOException; import java.text.DateFormat; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Collections; import java.util.Date; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; import java.util.Locale; -import java.util.Map; import java.util.Set; import java.util.TimeZone; @@ -51,10 +46,11 @@ 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; -import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.apache.hadoop.yarn.webapp.BadRequestException; @@ -63,7 +59,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.inject.Singleton; -/** REST end point for Timeline Reader */ +/** REST end point for Timeline Reader. */ @Private @Unstable @Singleton @@ -167,117 +163,6 @@ private static DateRange parseDateRange(String dateRange) } } - private static Set parseValuesStr(String str, String delimiter) { - if (str == null || str.isEmpty()) { - return null; - } - Set strSet = new HashSet(); - String[] strs = str.split(delimiter); - for (String aStr : strs) { - strSet.add(aStr.trim()); - } - return strSet; - } - - @SuppressWarnings("unchecked") - private static void parseKeyValues(Map map, String str, - String pairsDelim, String keyValuesDelim, boolean stringValue, - boolean multipleValues) { - String[] pairs = str.split(pairsDelim); - for (String pair : pairs) { - if (pair == null || pair.trim().isEmpty()) { - continue; - } - String[] pairStrs = pair.split(keyValuesDelim); - if (pairStrs.length < 2) { - continue; - } - if (!stringValue) { - try { - Object value = - GenericObjectMapper.OBJECT_READER.readValue(pairStrs[1].trim()); - map.put(pairStrs[0].trim(), (T) value); - } catch (IOException e) { - map.put(pairStrs[0].trim(), (T) pairStrs[1].trim()); - } - } else { - String key = pairStrs[0].trim(); - if (multipleValues) { - Set values = new HashSet(); - for (int i = 1; i < pairStrs.length; i++) { - values.add(pairStrs[i].trim()); - } - map.put(key, (T) values); - } else { - map.put(key, (T) pairStrs[1].trim()); - } - } - } - } - - private static Map> parseKeyStrValuesStr(String str, - String pairsDelim, String keyValuesDelim) { - if (str == null) { - return null; - } - Map> map = new HashMap>(); - parseKeyValues(map, str,pairsDelim, keyValuesDelim, true, true); - return map; - } - - private static Map parseKeyStrValueStr(String str, - String pairsDelim, String keyValDelim) { - if (str == null) { - return null; - } - Map map = new HashMap(); - parseKeyValues(map, str, pairsDelim, keyValDelim, true, false); - return map; - } - - private static Map parseKeyStrValueObj(String str, - String pairsDelim, String keyValDelim) { - if (str == null) { - return null; - } - Map map = new HashMap(); - parseKeyValues(map, str, pairsDelim, keyValDelim, false, false); - return map; - } - - private static EnumSet parseFieldsStr(String str, String delimiter) { - if (str == null) { - return null; - } - String[] strs = str.split(delimiter); - EnumSet fieldList = EnumSet.noneOf(Field.class); - for (String s : strs) { - fieldList.add(Field.valueOf(s.trim().toUpperCase())); - } - return fieldList; - } - - private static Long parseLongStr(String str) { - return str == null ? null : Long.parseLong(str.trim()); - } - - private static String parseStr(String str) { - return str == null ? null : str.trim(); - } - - private static UserGroupInformation getUser(HttpServletRequest req) { - String remoteUser = req.getRemoteUser(); - UserGroupInformation callerUGI = null; - if (remoteUser != null) { - callerUGI = UserGroupInformation.createRemoteUser(remoteUser); - } - return callerUGI; - } - - private static String getUserName(UserGroupInformation callerUGI) { - return ((callerUGI != null) ? callerUGI.getUserName().trim() : ""); - } - private TimelineReaderManager getTimelineReaderManager() { return (TimelineReaderManager) ctxt.getAttribute(TimelineReaderServer.TIMELINE_READER_MANAGER_ATTR); @@ -294,6 +179,10 @@ private static void handleException(Exception e, String url, long startTime, } else if (e instanceof IllegalArgumentException) { throw new BadRequestException(e.getMessage() == null ? "Requested Invalid Field." : e.getMessage()); + } else if (e instanceof NotFoundException) { + throw (NotFoundException)e; + } else if (e instanceof BadRequestException) { + throw (BadRequestException)e; } else { LOG.error("Error while processing REST request", e); throw new WebApplicationException(e, @@ -303,6 +192,11 @@ private static void handleException(Exception e, String url, long startTime, /** * Return the description of the timeline reader web services. + * + * @param req Servlet request. + * @param res Servlet response. + * + * @return information about the cluster including timeline version. */ @GET @Produces(MediaType.APPLICATION_JSON) @@ -314,11 +208,197 @@ public TimelineAbout about( } /** + * Return a single entity for a given entity type and UID which is a delimited + * string containing clusterid, userid, flow name, flowrun id and app id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name, + * flowrun id and app id which are extracted from UID and then used to + * query backend(Mandatory path param). + * @param limit Number of entities to return(Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param modifiedTimeStart If specified, matched entities should not be + * modified before this timestamp(Optional query param). + * @param modifiedTimeEnd If specified, matched entities should not be + * modified after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format [entitytype]:[entityid1]:[entityid2]... + * (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as + * metricfilters=metricid1, metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * 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 and modified + * 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, + * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/app-uid/{uid}/entities/{entitytype}") + @Produces(MediaType.APPLICATION_JSON) + public Set getEntities( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @PathParam("entitytype") String entityType, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("modifiedtimestart") String modifiedTimeStart, + @QueryParam("modifiedtimeend") String modifiedTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("fields") String fields) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.APPLICATION_UID.decodeUID(uId); + if (context == null) { + throw new BadRequestException("Incorrect UID " + uId); + } + entities = timelineReaderManager.getEntities( + TimelineReaderUtils.parseStr(context.getUserId()), + TimelineReaderUtils.parseStr(context.getClusterId()), + TimelineReaderUtils.parseStr(context.getFlowName()), + context.getFlowRunId(), + TimelineReaderUtils.parseStr(context.getAppId()), + TimelineReaderUtils.parseStr(entityType), + TimelineReaderUtils.parseLongStr(limit), + TimelineReaderUtils.parseLongStr(createdTimeStart), + TimelineReaderUtils.parseLongStr(createdTimeEnd), + TimelineReaderUtils.parseLongStr(modifiedTimeStart), + TimelineReaderUtils.parseLongStr(modifiedTimeEnd), + TimelineReaderUtils.parseKeyStrValuesStr(relatesTo, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValuesStr(isRelatedTo, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValueObj(infofilters, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValueStr(conffilters, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseValuesStr(metricfilters, COMMA_DELIMITER), + TimelineReaderUtils.parseValuesStr(eventfilters, COMMA_DELIMITER), + TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); + } catch (Exception e) { + e.printStackTrace(); + handleException(e, url, startTime, + "createdTime or modifiedTime start/end or limit or flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** * Return a set of entities that match the given parameters. Cluster ID is not - * provided by client so default cluster ID has to be taken. + * provided by client so default cluster ID has to be taken. If userid, flow + * name and flowrun id which are optional query parameters are not specified, + * they will be queried based on app id and default cluster id from the flow + * context information stored in underlying storage implementation. If number + * of matching entities are more than the limit, most recent entities till the + * limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to which the entities to be queried belong to( + * Mandatory path param). + * @param entityType Type of entities(Mandatory path param). + * @param userId User id which should match for the entities(Optional query + * param) + * @param flowName Flow name which should match for the entities(Optional + * query param). + * @param flowRunId Run id which should match for the entities(Optional query + * param). + * @param limit Number of entities to return(Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param modifiedTimeStart If specified, matched entities should not be + * modified before this timestamp(Optional query param). + * @param modifiedTimeEnd If specified, matched entities should not be + * modified after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format [entitytype]:[entityid1]:[entityid2]... + * (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as + * metricfilters=metricid1, metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * 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 and modified + * 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * If flow context information cannot be retrieved, HTTP 404(Not Found) + * is returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/entities/{appid}/{entitytype}/") + @Path("/apps/{appid}/entities/{entitytype}") @Produces(MediaType.APPLICATION_JSON) public Set getEntities( @Context HttpServletRequest req, @@ -339,7 +419,7 @@ public TimelineAbout about( @QueryParam("conffilters") String conffilters, @QueryParam("metricfilters") String metricfilters, @QueryParam("eventfilters") String eventfilters, - @QueryParam("fields") String fields) { + @QueryParam("fields") String fields) { return getEntities(req, res, null, appId, entityType, userId, flowName, flowRunId, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart, modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters, @@ -347,10 +427,73 @@ public TimelineAbout about( } /** - * Return a set of entities that match the given parameters. + * Return a set of entities that match the given parameters. If userid, flow + * name and flowrun id which are optional query parameters are not specified, + * they will be queried based on app id and cluster id from the flow context + * information stored in underlying storage implementation. If number of + * matching entities are more than the limit, most recent entities till the + * limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the entities to be queried belong to( + * Mandatory path param). + * @param appId Application id to which the entities to be queried belong to( + * Mandatory path param). + * @param entityType Type of entities(Mandatory path param). + * @param userId User id which should match for the entities(Optional query + * param) + * @param flowName Flow name which should match for the entities(Optional + * query param). + * @param flowRunId Run id which should match for the entities(Optional query + * param). + * @param limit Number of entities to return(Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param modifiedTimeStart If specified, matched entities should not be + * modified before this timestamp(Optional query param). + * @param modifiedTimeEnd If specified, matched entities should not be + * modified after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format [entitytype]:[entityid1]:[entityid2]... + * (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as + * metricfilters=metricid1, metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * 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 and modified + * 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * If flow context information cannot be retrieved, HTTP 404(Not Found) + * is returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/entities/{clusterid}/{appid}/{entitytype}/") + @Path("/clusters/{clusterid}/apps/{appid}/entities/{entitytype}") @Produces(MediaType.APPLICATION_JSON) public Set getEntities( @Context HttpServletRequest req, @@ -376,26 +519,37 @@ public TimelineAbout about( String url = req.getRequestURI() + (req.getQueryString() == null ? "" : QUERY_STRING_SEP + req.getQueryString()); - UserGroupInformation callerUGI = getUser(req); - LOG.info("Received URL " + url + " from user " + getUserName(callerUGI)); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); long startTime = Time.monotonicNow(); init(res); TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); Set entities = null; try { entities = timelineReaderManager.getEntities( - parseStr(userId), parseStr(clusterId), parseStr(flowName), - parseLongStr(flowRunId), parseStr(appId), parseStr(entityType), - parseLongStr(limit), parseLongStr(createdTimeStart), - parseLongStr(createdTimeEnd), parseLongStr(modifiedTimeStart), - parseLongStr(modifiedTimeEnd), - 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), - parseFieldsStr(fields, COMMA_DELIMITER)); + TimelineReaderUtils.parseStr(userId), + TimelineReaderUtils.parseStr(clusterId), + TimelineReaderUtils.parseStr(flowName), + TimelineReaderUtils.parseLongStr(flowRunId), + TimelineReaderUtils.parseStr(appId), + TimelineReaderUtils.parseStr(entityType), + TimelineReaderUtils.parseLongStr(limit), + TimelineReaderUtils.parseLongStr(createdTimeStart), + TimelineReaderUtils.parseLongStr(createdTimeEnd), + TimelineReaderUtils.parseLongStr(modifiedTimeStart), + TimelineReaderUtils.parseLongStr(modifiedTimeEnd), + TimelineReaderUtils.parseKeyStrValuesStr(relatesTo, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValuesStr(isRelatedTo, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValueObj(infofilters, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValueStr(conffilters, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseValuesStr(metricfilters, COMMA_DELIMITER), + TimelineReaderUtils.parseValuesStr(eventfilters, COMMA_DELIMITER), + TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); } catch (Exception e) { handleException(e, url, startTime, "createdTime or modifiedTime start/end or limit or flowrunid"); @@ -410,11 +564,108 @@ public TimelineAbout about( } /** + * Return a single entity for given UID which is a delimited string containing + * clusterid, userid, flow name, flowrun id, app id, entity type and entityid. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name, + * flowrun id, app id, entity type and entity id which are extracted from + * 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 and modified + * 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, + * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned. + * If entity for the given 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. + */ + @GET + @Path("/entity-uid/{uid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getEntity( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("fields") String fields) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uId); + 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(), + TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but entity not found" + " (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("Timeline entity with uid: " + uId + + "is not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** * Return a single entity of the given entity type and Id. Cluster ID is not - * provided by client so default cluster ID has to be taken. + * provided by client so default cluster ID has to be taken. If userid, flow + * name and flowrun id which are optional query parameters are not specified, + * they will be queried based on app id and default cluster id from the flow + * context information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to which the entity to be queried belongs to( + * Mandatory path param). + * @param entityType Type of entity(Mandatory path param). + * @param entityId Id of the entity to be fetched(Mandatory path param). + * @param userId User id which should match for the entity(Optional query + * param). + * @param flowName Flow name which should match for the entity(Optional query + * param). + * @param flowRunId Run id which should match for the entity(Optional query + * 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 and modified + * 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * If flow context information cannot be retrieved or entity for the given + * 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. */ @GET - @Path("/entity/{appid}/{entitytype}/{entityid}/") + @Path("/apps/{appid}/entities/{entitytype}/{entityid}/") @Produces(MediaType.APPLICATION_JSON) public TimelineEntity getEntity( @Context HttpServletRequest req, @@ -431,10 +682,42 @@ public TimelineEntity getEntity( } /** - * Return a single entity of the given entity type and Id. + * Return a single entity of the given entity type and Id. If userid, flowname + * and flowrun id which are optional query parameters are not specified, they + * will be queried based on app id and cluster id from the flow context + * information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the entity to be queried belongs to( + * Mandatory path param). + * @param appId Application id to which the entity to be queried belongs to( + * Mandatory path param). + * @param entityType Type of entity(Mandatory path param). + * @param entityId Id of the entity to be fetched(Mandatory path param). + * @param userId User id which should match for the entity(Optional query + * param). + * @param flowName Flow name which should match for the entity(Optional query + * param). + * @param flowRunId Run id which should match for the entity(Optional query + * 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 and modified + * 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * If flow context information cannot be retrieved or entity for the given + * 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. */ @GET - @Path("/entity/{clusterid}/{appid}/{entitytype}/{entityid}/") + @Path("/clusters/{clusterid}/apps/{appid}/entities/{entitytype}/{entityid}/") @Produces(MediaType.APPLICATION_JSON) public TimelineEntity getEntity( @Context HttpServletRequest req, @@ -450,17 +733,23 @@ public TimelineEntity getEntity( String url = req.getRequestURI() + (req.getQueryString() == null ? "" : QUERY_STRING_SEP + req.getQueryString()); - UserGroupInformation callerUGI = getUser(req); - LOG.info("Received URL " + url + " from user " + getUserName(callerUGI)); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); long startTime = Time.monotonicNow(); init(res); TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); TimelineEntity entity = null; + String type = TimelineReaderUtils.parseStr(entityType); + String id = TimelineReaderUtils.parseStr(entityId); try { entity = timelineReaderManager.getEntity( - parseStr(userId), parseStr(clusterId), parseStr(flowName), - parseLongStr(flowRunId), parseStr(appId), parseStr(entityType), - parseStr(entityId), parseFieldsStr(fields, COMMA_DELIMITER)); + TimelineReaderUtils.parseStr(userId), + TimelineReaderUtils.parseStr(clusterId), + TimelineReaderUtils.parseStr(flowName), + TimelineReaderUtils.parseLongStr(flowRunId), + TimelineReaderUtils.parseStr(appId), type, id, + TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -468,8 +757,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: " + parseStr(entityId) + - ", type: " + parseStr(entityType) + " } is not found"); + throw new NotFoundException("Timeline entity {id: " + id + ", type: " + + type + " } is not found"); } LOG.info("Processed URL " + url + " (Took " + (endTime - startTime) + " ms.)"); @@ -477,50 +766,153 @@ public TimelineEntity getEntity( } /** - * Return a single flow run for the given user, flow id and run id. + * Return a single flow run for given UID which is a delimited string + * containing clusterid, userid, flow name and flowrun id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name and + * flowrun id which are extracted from UID and then used to query backend + * (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, + * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned. + * If flow run for the given flow run id cannot be found, HTTP 404 + * (Not Found) is returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/run-uid/{uid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getFlowRun( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.FLOWRUN_UID.decodeUID(uId); + 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); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but flowrun not found (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("Flowrun with uid: " + uId + "is not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** + * Return a single flow run for the given user, flow name and run id. * Cluster ID is not provided by client so default cluster ID has to be taken. + * + * @param req Servlet request. + * @param res Servlet response. + * @param userId User id representing the user who ran the flow run(Mandatory + * path param). + * @param flowName Flow name to which the flow run to be queried belongs to( + * Mandatory path param). + * @param flowRunId Id of the flow run to be queried(Mandatory path param). + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * {@link 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. + * If flow run for the given flow run id cannot be found, HTTP 404 + * (Not Found) is returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flowrun/{userid}/{flowname}/{flowrunid}/") + @Path("/users/{userid}/flows/{flowname}/runs/{flowrunid}/") @Produces(MediaType.APPLICATION_JSON) public TimelineEntity getFlowRun( @Context HttpServletRequest req, @Context HttpServletResponse res, @PathParam("userid") String userId, @PathParam("flowname") String flowName, - @PathParam("flowrunid") String flowRunId, - @QueryParam("fields") String fields) { - return getFlowRun(req, res, userId, null, flowName, flowRunId, fields); + @PathParam("flowrunid") String flowRunId) { + return getFlowRun(req, res, null, userId, flowName, flowRunId); } /** - * Return a single flow run for the given user, cluster, flow id and run id. + * Return a single flow run for the given user, cluster, flow name and run id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the flow run to be queried belong to( + * Mandatory path param). + * @param userId User id representing the user who ran the flow run(Mandatory + * path param). + * @param flowName Flow name to which the flow run to be queried belongs to( + * Mandatory path param). + * @param flowRunId Id of the flow run to be queried(Mandatory path param). + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * {@link 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. + * If flow run for the given flow run id cannot be found, HTTP 404 + * (Not Found) is returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flowrun/{userid}/{clusterid}/{flowname}/{flowrunid}/") + @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/" + + "runs/{flowrunid}/") @Produces(MediaType.APPLICATION_JSON) public TimelineEntity getFlowRun( @Context HttpServletRequest req, @Context HttpServletResponse res, - @PathParam("userid") String userId, @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, @PathParam("flowname") String flowName, - @PathParam("flowrunid") String flowRunId, - @QueryParam("fields") String fields) { + @PathParam("flowrunid") String flowRunId) { String url = req.getRequestURI() + (req.getQueryString() == null ? "" : QUERY_STRING_SEP + req.getQueryString()); - UserGroupInformation callerUGI = getUser(req); - LOG.info("Received URL " + url + " from user " + getUserName(callerUGI)); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); long startTime = Time.monotonicNow(); init(res); TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); TimelineEntity entity = null; try { - entity = timelineReaderManager.getEntity(parseStr(userId), - parseStr(clusterId), parseStr(flowName), parseLongStr(flowRunId), null, - TimelineEntityType.YARN_FLOW_RUN.toString(), null, - parseFieldsStr(fields, COMMA_DELIMITER)); + entity = timelineReaderManager.getEntity( + TimelineReaderUtils.parseStr(userId), + TimelineReaderUtils.parseStr(clusterId), + TimelineReaderUtils.parseStr(flowName), + TimelineReaderUtils.parseLongStr(flowRunId), + null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -528,8 +920,9 @@ public TimelineEntity getFlowRun( if (entity == null) { LOG.info("Processed URL " + url + " but flowrun not found (Took " + (endTime - startTime) + " ms.)"); - throw new NotFoundException("Flow run {flow id: " + parseStr(flowName) + - ", run id: " + parseLongStr(flowRunId) + " } is not found"); + throw new NotFoundException("Flow run {flow name: " + + TimelineReaderUtils.parseStr(flowName) + ", run id: " + + TimelineReaderUtils.parseLongStr(flowRunId) + " } is not found"); } LOG.info("Processed URL " + url + " (Took " + (endTime - startTime) + " ms.)"); @@ -537,11 +930,111 @@ public TimelineEntity getFlowRun( } /** - * Return a set of flows runs for the given user and flow id. + * Return a list of flow runs for given UID which is a delimited string + * containing clusterid, userid and flow name. + * + * @param req Servlet request. + * @param res Servlet response. + * @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). + * @param createdTimeEnd If specified, matched flow runs should not be created + * after this timestamp(Optional query param). + * @param fields Specifies which fields to retrieve, see {@link Field}. + * All fields will be retrieved if fields=ALL. Fields other than METRICS + * have no meaning for this REST endpoint. If not specified, all fields + * 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, + * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/flow-uid/{uid}/runs/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowRuns( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("fields") String fields) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.FLOW_UID.decodeUID(uId); + 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(), + TimelineReaderUtils.parseLongStr(limit), + TimelineReaderUtils.parseLongStr(createdTimeStart), + TimelineReaderUtils.parseLongStr(createdTimeEnd), null, + null, null, null, null, null, null, null, + TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); + } catch (Exception e) { + handleException(e, url, startTime, "createdTime start/end or limit"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** + * Return a set of flows runs for the given user and flow name. * Cluster ID is not provided by client so default cluster ID has to be taken. + * + * @param req Servlet request. + * @param res Servlet response. + * @param userId User id representing the user who ran the flow runs( + * 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). + * @param createdTimeEnd If specified, matched flow runs should not be created + * after this timestamp(Optional query param). + * @param fields Specifies which fields to retrieve, see {@link Field}. + * All fields will be retrieved if fields=ALL. Fields other than METRICS + * have no meaning for this REST endpoint. If not specified, all fields + * 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flowruns/{userid}/{flowname}/") + @Path("/users/{userid}/flows/{flowname}/runs/") @Produces(MediaType.APPLICATION_JSON) public Set getFlowRuns( @Context HttpServletRequest req, @@ -552,21 +1045,47 @@ public TimelineEntity getFlowRun( @QueryParam("createdtimestart") String createdTimeStart, @QueryParam("createdtimeend") String createdTimeEnd, @QueryParam("fields") String fields) { - return getFlowRuns(req, res, userId, null, flowName, limit, createdTimeStart, - createdTimeEnd, fields); + return getFlowRuns(req, res, null, userId, flowName, limit, + createdTimeStart, createdTimeEnd, fields); } /** - * Return a set of flow runs for the given user, cluster and flow id. + * Return a set of flows runs for the given cluster, user and flow name. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the flow runs to be queried belong to( + * Mandatory path param). + * @param userId User id representing the user who ran the flow runs( + * 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). + * @param createdTimeEnd If specified, matched flow runs should not be created + * after this timestamp(Optional query param). + * @param fields Specifies which fields to retrieve, see {@link Field}. + * All fields will be retrieved if fields=ALL. Fields other than METRICS + * have no meaning for this REST endpoint. If not specified, all fields + * 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flowruns/{userid}/{clusterid}/{flowname}/") + @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/runs/") @Produces(MediaType.APPLICATION_JSON) public Set getFlowRuns( @Context HttpServletRequest req, @Context HttpServletResponse res, - @PathParam("userid") String userId, @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, @PathParam("flowname") String flowName, @QueryParam("limit") String limit, @QueryParam("createdtimestart") String createdTimeStart, @@ -575,19 +1094,24 @@ public TimelineEntity getFlowRun( String url = req.getRequestURI() + (req.getQueryString() == null ? "" : QUERY_STRING_SEP + req.getQueryString()); - UserGroupInformation callerUGI = getUser(req); - LOG.info("Received URL " + url + " from user " + getUserName(callerUGI)); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); long startTime = Time.monotonicNow(); init(res); TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); Set entities = null; try { entities = timelineReaderManager.getEntities( - parseStr(userId), parseStr(clusterId), parseStr(flowName), null, null, - TimelineEntityType.YARN_FLOW_RUN.toString(), parseLongStr(limit), - parseLongStr(createdTimeStart), parseLongStr(createdTimeEnd), null, + TimelineReaderUtils.parseStr(userId), + TimelineReaderUtils.parseStr(clusterId), + TimelineReaderUtils.parseStr(flowName), null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), + TimelineReaderUtils.parseLongStr(limit), + TimelineReaderUtils.parseLongStr(createdTimeStart), + TimelineReaderUtils.parseLongStr(createdTimeEnd), null, null, null, null, null, null, null, null, - parseFieldsStr(fields, COMMA_DELIMITER)); + TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit"); } @@ -601,20 +1125,35 @@ public TimelineEntity getFlowRun( } /** - * Return a list of flows. Cluster ID is not provided by client so default - * cluster ID has to be taken. daterange, if specified is given as - * "[startdate]-[enddate]"(i.e. start and end date separated by -) or - * single date. Dates are interpreted in yyyyMMdd format and are assumed to - * be in GMT. If a single date is specified, all flows active on that date are - * returned. If both startdate and enddate is given, all flows active between - * start and end date will be returned. If only startdate is given, flows - * active on and after startdate are returned. If only enddate is given, flows - * active on and before enddate are returned. - * For example : - * "daterange=20150711" returns flows active on 20150711. - * "daterange=20150711-20150714" returns flows active between these 2 dates. - * "daterange=20150711-" returns flows active on and after 20150711. - * "daterange=-20150711" returns flows active on and before 20150711. + * Return a list of active flows. Cluster ID is not provided by client so + * default cluster ID has to be taken. + * + * @param req Servlet request. + * @param res Servlet response. + * @param limit Number of flows to return(Optional query param). + * @param dateRange If specified is given as "[startdate]-[enddate]"(i.e. + * start and end date separated by "-") or single date. Dates are + * interpreted in yyyyMMdd format and are assumed to be in GMT(Optional + * query param). + * If a single date is specified, all flows active on that date are + * returned. If both startdate and enddate is given, all flows active + * between start and end date will be returned. If only startdate is + * given, flows active on and after startdate are returned. If only + * enddate is given, flows active on and before enddate are returned. + * For example : + * "daterange=20150711" returns flows active on 20150711. + * "daterange=20150711-20150714" returns flows active between these + * 2 dates. + * "daterange=20150711-" returns flows active on and after 20150711. + * "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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET @Path("/flows/") @@ -623,41 +1162,57 @@ public TimelineEntity getFlowRun( @Context HttpServletRequest req, @Context HttpServletResponse res, @QueryParam("limit") String limit, - @QueryParam("daterange") String dateRange, - @QueryParam("fields") String fields) { - return getFlows(req, res, null, limit, dateRange, fields); + @QueryParam("daterange") String dateRange) { + return getFlows(req, res, null, limit, dateRange); } /** - * Return a list of flows for a given cluster id. daterange, if specified is - * given as "[startdate]-[enddate]"(i.e. start and end date separated by -) or - * single date. Dates are interpreted in yyyyMMdd format and are assumed to - * be in GMT. If a single date is specified, all flows active on that date are - * returned. If both startdate and enddate is given, all flows active between - * start and end date will be returned. If only startdate is given, flows - * active on and after startdate are returned. If only enddate is given, flows - * active on and before enddate are returned. - * For example : - * "daterange=20150711" returns flows active on 20150711. - * "daterange=20150711-20150714" returns flows active between these 2 dates. - * "daterange=20150711-" returns flows active on and after 20150711. - * "daterange=-20150711" returns flows active on and before 20150711. + * Return a list of active flows for a given cluster id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the flows to be queried belong to( + * Mandatory path param). + * @param limit Number of flows to return(Optional query param). + * @param dateRange If specified is given as "[startdate]-[enddate]"(i.e. + * start and end date separated by "-") or single date. Dates are + * interpreted in yyyyMMdd format and are assumed to be in GMT(Optional + * query param). + * If a single date is specified, all flows active on that date are + * returned. If both startdate and enddate is given, all flows active + * between start and end date will be returned. If only startdate is + * given, flows active on and after startdate are returned. If only + * enddate is given, flows active on and before enddate are returned. + * For example : + * "daterange=20150711" returns flows active on 20150711. + * "daterange=20150711-20150714" returns flows active between these + * 2 dates. + * "daterange=20150711-" returns flows active on and after 20150711. + * "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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flows/{clusterid}/") + @Path("/clusters/{clusterid}/flows/") @Produces(MediaType.APPLICATION_JSON) public Set getFlows( @Context HttpServletRequest req, @Context HttpServletResponse res, @PathParam("clusterid") String clusterId, @QueryParam("limit") String limit, - @QueryParam("daterange") String dateRange, - @QueryParam("fields") String fields) { + @QueryParam("daterange") String dateRange) { String url = req.getRequestURI() + (req.getQueryString() == null ? "" : QUERY_STRING_SEP + req.getQueryString()); - UserGroupInformation callerUGI = getUser(req); - LOG.info("Received URL " + url + " from user " + getUserName(callerUGI)); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); long startTime = Time.monotonicNow(); init(res); TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); @@ -665,10 +1220,11 @@ public TimelineEntity getFlowRun( try { DateRange range = parseDateRange(dateRange); entities = timelineReaderManager.getEntities( - null, parseStr(clusterId), null, null, null, - TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), parseLongStr(limit), + null, TimelineReaderUtils.parseStr(clusterId), null, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), + TimelineReaderUtils.parseLongStr(limit), range.dateStart, range.dateEnd, null, null, null, null, null, null, - null, null, parseFieldsStr(fields, COMMA_DELIMITER)); + null, null, null); } catch (Exception e) { handleException(e, url, startTime, "limit"); } @@ -682,11 +1238,105 @@ public TimelineEntity getFlowRun( } /** - * Return a single app for given app id. Cluster ID is not provided by - * client so default cluster ID has to be taken. + * Return a single app for given UID which is a delimited string containing + * clusterid, userid, flow name, flowrun id and app id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name, flow + * run id and app id which are extracted from UID and then used to query + * 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), + * app id, app created time and modified 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, + * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned. + * If app for the given 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. + */ + @GET + @Path("/app-uid/{uid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getApp( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("fields") String fields) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.APPLICATION_UID.decodeUID(uId); + 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, TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but app not found" + " (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("App with uid " + uId + " not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** + * Return a single app for given app id. Cluster ID is not provided by client + * client so default cluster ID has to be taken. If userid, flow name and flow + * run id which are optional query parameters are not specified, they will be + * queried based on app id and cluster id from the flow context information + * stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to be queried(Mandatory path param). + * @param flowName Flow name which should match for the app(Optional query + * param). + * @param flowRunId Run id which should match for the app(Optional query + * param). + * @param userId User id which should match for the app(Optional query param). + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id, app created time and modified 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * If flow context information cannot be retrieved or app for the given + * 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. */ @GET - @Path("/app/{appid}/") + @Path("/apps/{appid}/") @Produces(MediaType.APPLICATION_JSON) public TimelineEntity getApp( @Context HttpServletRequest req, @@ -700,10 +1350,39 @@ public TimelineEntity getApp( } /** - * Return a single app for given cluster id and app id. + * Return a single app for given cluster id and app id. If userid, flow name + * and flowrun id which are optional query parameters are not specified, they + * will be queried based on app id and cluster id from the flow context + * information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the app to be queried belong to( + * Mandatory path param). + * @param appId Application id to be queried(Mandatory path param). + * @param flowName Flow name which should match for the app(Optional query + * param). + * @param flowRunId Run id which should match for the app(Optional query + * param). + * @param userId User id which should match for the app(Optional query param). + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 4 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id, app created time and modified 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * If flow context information cannot be retrieved or app for the given + * 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. */ @GET - @Path("/app/{clusterid}/{appid}/") + @Path("/clusters/{clusterid}/apps/{appid}/") @Produces(MediaType.APPLICATION_JSON) public TimelineEntity getApp( @Context HttpServletRequest req, @@ -717,17 +1396,22 @@ public TimelineEntity getApp( String url = req.getRequestURI() + (req.getQueryString() == null ? "" : QUERY_STRING_SEP + req.getQueryString()); - UserGroupInformation callerUGI = getUser(req); - LOG.info("Received URL " + url + " from user " + getUserName(callerUGI)); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); long startTime = Time.monotonicNow(); init(res); TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); TimelineEntity entity = null; try { - entity = timelineReaderManager.getEntity(parseStr(userId), - parseStr(clusterId), parseStr(flowName), parseLongStr(flowRunId), - parseStr(appId), TimelineEntityType.YARN_APPLICATION.toString(), null, - parseFieldsStr(fields, COMMA_DELIMITER)); + entity = timelineReaderManager.getEntity( + TimelineReaderUtils.parseStr(userId), + TimelineReaderUtils.parseStr(clusterId), + TimelineReaderUtils.parseStr(flowName), + TimelineReaderUtils.parseLongStr(flowRunId), + TimelineReaderUtils.parseStr(appId), + TimelineEntityType.YARN_APPLICATION.toString(), null, + TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -743,13 +1427,190 @@ public TimelineEntity getApp( } /** - * Return a list of apps for given user, flow id and flow run id. Cluster ID + * Return a list of apps for given UID which is a delimited string containing + * clusterid, userid, flow name and flowrun id. If number of matching apps are + * more than the limit, most recent apps till the limit is reached, will be + * returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name and + * flowrun id which are extracted from UID and then used to query backend. + * (Mandatory path param). + * @param limit Number of apps to return(Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param modifiedTimeStart If specified, matched apps should not be + * modified before this timestamp(Optional query param). + * @param modifiedTimeEnd If specified, matched apps should not be modified + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * 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), + * app id, app created time and modified 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, + * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/run-uid/{uid}/apps") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowRunApps( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("modifiedtimestart") String modifiedTimeStart, + @QueryParam("modifiedtimeend") String modifiedTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("fields") String fields) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = TimelineReaderUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.FLOWRUN_UID.decodeUID(uId); + if (context == null) { + throw new BadRequestException("Incorrect UID " + uId); + } + entities = timelineReaderManager.getEntities( + TimelineReaderUtils.parseStr(context.getUserId()), + TimelineReaderUtils.parseStr(context.getClusterId()), + TimelineReaderUtils.parseStr(context.getFlowName()), + context.getFlowRunId(), + TimelineReaderUtils.parseStr(context.getAppId()), + TimelineEntityType.YARN_APPLICATION.toString(), + TimelineReaderUtils.parseLongStr(limit), + TimelineReaderUtils.parseLongStr(createdTimeStart), + TimelineReaderUtils.parseLongStr(createdTimeEnd), + TimelineReaderUtils.parseLongStr(modifiedTimeStart), + TimelineReaderUtils.parseLongStr(modifiedTimeEnd), + TimelineReaderUtils.parseKeyStrValuesStr(relatesTo, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValuesStr(isRelatedTo, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValueObj(infofilters, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseKeyStrValueStr(conffilters, COMMA_DELIMITER, + COLON_DELIMITER), + TimelineReaderUtils.parseValuesStr(metricfilters, COMMA_DELIMITER), + TimelineReaderUtils.parseValuesStr(eventfilters, COMMA_DELIMITER), + TimelineReaderUtils.parseFieldsStr(fields, COMMA_DELIMITER)); + } catch (Exception e) { + handleException(e, url, startTime, + "createdTime or modifiedTime start/end or limit or flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** + * Return a list of apps for given user, flow name and flow run id. Cluster ID * is not provided by client so default cluster ID has to be taken. If number * of matching apps are more than the limit, most recent apps till the limit * is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param userId User id which should match for the apps(Mandatory path param) + * @param flowName Flow name which should match for the apps(Mandatory path + * param). + * @param flowRunId Run id which should match for the apps(Mandatory path + * param). + * @param limit Number of apps to return(Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param modifiedTimeStart If specified, matched apps should not be + * modified before this timestamp(Optional query param). + * @param modifiedTimeEnd If specified, matched apps should not be modified + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * 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), + * app id, app created time and modified 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flowrunapps/{userid}/{flowname}/{flowrunid}/") + @Path("/users/{userid}/flows/{flowname}/runs/{flowrunid}/apps/") @Produces(MediaType.APPLICATION_JSON) public Set getFlowRunApps( @Context HttpServletRequest req, @@ -777,18 +1638,72 @@ public TimelineEntity getApp( } /** - * Return a list of apps for a given user, cluster id, flow id and flow run + * Return a list of apps for a given user, cluster id, flow name and flow run * id. If number of matching apps are more than the limit, most recent apps * till the limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the apps to be queried belong to + * (Mandatory path param). + * @param userId User id which should match for the apps(Mandatory path param) + * @param flowName Flow name which should match for the apps(Mandatory path + * param). + * @param flowRunId Run id which should match for the apps(Mandatory path + * param). + * @param limit Number of apps to return(Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param modifiedTimeStart If specified, matched apps should not be + * modified before this timestamp(Optional query param). + * @param modifiedTimeEnd If specified, matched apps should not be modified + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * 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), + * app id, app created time and modified 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flowrunapps/{userid}/{clusterid}/{flowname}/{flowrunid}/") + @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/runs/" + + "{flowrunid}/apps/") @Produces(MediaType.APPLICATION_JSON) public Set getFlowRunApps( @Context HttpServletRequest req, @Context HttpServletResponse res, - @PathParam("userid") String userId, @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, @PathParam("flowname") String flowName, @PathParam("flowrunid") String flowRunId, @QueryParam("limit") String limit, @@ -811,13 +1726,62 @@ public TimelineEntity getApp( } /** - * Return a list of apps for given user and flow id. Cluster ID is not + * Return a list of apps for given user and flow name. Cluster ID is not * provided by client so default cluster ID has to be taken. If number of * matching apps are more than the limit, most recent apps till the limit is * reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param userId User id which should match for the apps(Mandatory path param) + * @param flowName Flow name which should match for the apps(Mandatory path + * param). + * @param limit Number of apps to return(Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param modifiedTimeStart If specified, matched apps should not be + * modified before this timestamp(Optional query param). + * @param modifiedTimeEnd If specified, matched apps should not be modified + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * 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), + * app id, app created time and modified 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flowapps/{userid}/{flowname}/") + @Path("/users/{userid}/flows/{flowname}/apps/") @Produces(MediaType.APPLICATION_JSON) public Set getFlowApps( @Context HttpServletRequest req, @@ -844,18 +1808,70 @@ public TimelineEntity getApp( } /** - * Return a list of apps for a given user, cluster id and flow id. If number + * Return a list of apps for a given user, cluster id and flow name. If number * of matching apps are more than the limit, most recent apps till the limit - * is reached, will be returned. + * is reached, will be returned. If number of matching apps are more than the + * limit, most recent apps till the limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the apps to be queried belong to + * (Mandatory path param). + * @param userId User id which should match for the apps(Mandatory path param) + * @param flowName Flow name which should match for the apps(Mandatory path + * param). + * @param limit Number of apps to return(Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param modifiedTimeStart If specified, matched apps should not be + * modified before this timestamp(Optional query param). + * @param modifiedTimeEnd If specified, matched apps should not be modified + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * 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), + * app id, app created time and modified 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, + * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned. + * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. */ @GET - @Path("/flowapps/{userid}/{clusterid}/{flowname}/") + @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/apps/") @Produces(MediaType.APPLICATION_JSON) public Set getFlowApps( @Context HttpServletRequest req, @Context HttpServletResponse res, - @PathParam("userid") String userId, @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, @PathParam("flowname") String flowName, @QueryParam("limit") String limit, @QueryParam("createdtimestart") String createdTimeStart, diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java new file mode 100644 index 0000000..8fa7679 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java @@ -0,0 +1,245 @@ +/** + * 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.List; + +/** + * Used for encoding/decoding UID which will be used for query by UI. + */ +public enum TimelineUIDConverter { + // Flow UID should contain cluster, user and flow name. + FLOW_UID { + @Override + public String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getUserId() == null || + context.getFlowName() == null) { + return null; + } + String[] flowNameTupleArr = {context.getClusterId(), context.getUserId(), + context.getFlowName()}; + return joinAndEscapeUIDParts(flowNameTupleArr); + } + + @Override + public TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List flowNameTupleList = splitUID(uId); + // Should have 3 parts i.e. cluster, user and flow name. + if (flowNameTupleList.size() != 3) { + return null; + } + return new TimelineReaderContext(flowNameTupleList.get(0), + flowNameTupleList.get(1), flowNameTupleList.get(2), null, + null, null, null); + } + }, + + // Flowrun UID should contain cluster, user, flow name and flowrun id. + FLOWRUN_UID{ + @Override + public String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getUserId() == null || + context.getFlowName() == null || context.getFlowRunId() == null) { + return null; + } + String[] flowRunTupleArr = {context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId().toString()}; + return joinAndEscapeUIDParts(flowRunTupleArr); + } + + @Override + public TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List flowRunTupleList = splitUID(uId); + // Should have 4 parts i.e. cluster, user, flow name and flowrun id. + if (flowRunTupleList.size() != 4) { + return null; + } + return new TimelineReaderContext(flowRunTupleList.get(0), + flowRunTupleList.get(1), flowRunTupleList.get(2), + Long.parseLong(flowRunTupleList.get(3)), null, null, null); + } + }, + + // Application UID should contain cluster, user, flow name, flowrun id + // and app id OR cluster and app id(i.e.without flow context info). + APPLICATION_UID{ + @Override + public String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getAppId() == null) { + return null; + } + if (context.getUserId() != null && context.getFlowName() != null && + context.getFlowRunId() != null) { + // Flow information exists. + String[] appTupleArr = {context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId().toString(), + context.getAppId()}; + return joinAndEscapeUIDParts(appTupleArr); + } else { + // Only cluster and app information exists. Flow info does not exist. + String[] appTupleArr = {context.getClusterId(), context.getAppId()}; + return joinAndEscapeUIDParts(appTupleArr); + } + } + + @Override + public TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List appTupleList = splitUID(uId); + // Should have 5 parts i.e. cluster, user, flow name, flowrun id + // and app id OR should have 2 parts i.e. cluster and app id. + if (appTupleList.size() == 5) { + // Flow information exists. + return new TimelineReaderContext(appTupleList.get(0), + appTupleList.get(1), appTupleList.get(2), + Long.parseLong(appTupleList.get(3)), appTupleList.get(4), + null, null); + } else if (appTupleList.size() == 2) { + // Flow information does not exist. + return new TimelineReaderContext(appTupleList.get(0), null, null, null, + appTupleList.get(1), null, null); + } else { + return null; + } + } + }, + + // Generic Entity UID should contain cluster, user, flow name, flowrun id, + // app id, entity type and entity id OR should contain cluster, appid, entity + // type and entity id(i.e.without flow context info). + GENERIC_ENTITY_UID { + @Override + public String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getAppId() == null || + context.getEntityType() == null || context.getEntityId() == null) { + return null; + } + if (context.getUserId() != null && context.getFlowName() != null && + context.getFlowRunId() != null) { + // Flow information exists. + String[] entityTupleArr = {context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId().toString(), + context.getAppId(), context.getEntityType(), context.getEntityId()}; + return joinAndEscapeUIDParts(entityTupleArr); + } else { + // Only entity and app information exists. Flow info does not exist. + String[] entityTupleArr = {context.getClusterId(), context.getAppId(), + context.getEntityType(), context.getEntityId()}; + return joinAndEscapeUIDParts(entityTupleArr); + } + } + + @Override + public TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List entityTupleList = splitUID(uId); + // Should have 7 parts i.e. cluster, user, flow name, flowrun id, app id, + // entity type and entity id OR should have 4 parts i.e. cluster, app id, + // entity type and entity id. + if (entityTupleList.size() == 7) { + // Flow information exists. + return new TimelineReaderContext(entityTupleList.get(0), + entityTupleList.get(1), entityTupleList.get(2), + Long.parseLong(entityTupleList.get(3)), entityTupleList.get(4), + entityTupleList.get(5), entityTupleList.get(6)); + } else if (entityTupleList.size() == 4) { + // Flow information does not exist. + return new TimelineReaderContext(entityTupleList.get(0), null, null, + null, entityTupleList.get(1), entityTupleList.get(2), + entityTupleList.get(3)); + } else { + return null; + } + } + }; + + /** + * Delimiter used for UID. + */ + private static final char UID_DELIMITER_CHAR = '!'; + + /** + * Escape Character used if delimiter or escape character itself is part of + * different components of UID. + */ + private static final char UID_ESCAPE_CHAR = '*'; + + /** + * Split UID using {@link #UID_DELIMITER_CHAR} and {@link #UID_ESCAPE_CHAR}. + * @param uid + * @return a list of different parts of UID split across delimiter. + * @throws IllegalArgumentException if UID is not properly escaped. + */ + private static List splitUID(String uid) + throws IllegalArgumentException { + return TimelineReaderUtils.split(uid, UID_DELIMITER_CHAR, UID_ESCAPE_CHAR); + } + + /** + * Join different parts of UID delimited by {@link #UID_DELIMITER_CHAR} with + * delimiter and escape character escaped using {@link #UID_ESCAPE_CHAR} if + * UID parts contain them. + * @param parts an array of UID parts to be joined. + * @return a string joined using the delimiter with escape and delimiter + * characters escaped if they are part of the string parts to be joined. + * Returns null if one of the parts is null. + */ + private static String joinAndEscapeUIDParts(String[] parts) { + return TimelineReaderUtils.joinAndEscapeStrings(parts, UID_DELIMITER_CHAR, + UID_ESCAPE_CHAR); + } + + /** + * Encodes UID depending on UID implementation. + * @param context + * @return UID represented as a string. + */ + public abstract String encodeUID(TimelineReaderContext context); + + /** + * Decodes UID depending on UID implementation. + * @param uId + * @return a {@link TimelineReaderContext} object if UID passed can be + * decoded, null otherwise. + * @throws Exception + */ + public abstract TimelineReaderContext decodeUID(String uId) throws Exception; +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java 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 56bd3a0..4b98a47 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java @@ -26,6 +26,8 @@ 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; @@ -88,9 +90,13 @@ * Specifies which fields of the entity object to retrieve(optional), see * {@link Field}. If null, retrieves 4 fields namely entity id, * entity type, entity created time and entity modified time. All - * entities will be returned if {@link Field#ALL} is specified. + * 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. + * If entityType is YARN_FLOW_RUN, entity returned is of type + * {@link FlowRunEntity}. + * For all other entity types, entity returned is of type + * {@link TimelineEntity}. * @throws IOException */ TimelineEntity getEntity(String userId, String clusterId, String flowName, @@ -174,12 +180,18 @@ TimelineEntity getEntity(String userId, String clusterId, String flowName, * Specifies which fields of the entity object to retrieve(optional), see * {@link Field}. If null, retrieves 4 fields namely entity id, * entity type, entity created time and entity modified time. All - * entities will be returned if {@link Field#ALL} is specified. + * fields will be returned if {@link Field#ALL} is specified. * @return A set of {@link TimelineEntity} instances of the given entity type * in the given context scope which matches the given predicates * ordered by created time, descending. Each entity will only contain the * metadata(id, type, created and modified times) plus the given fields to * retrieve. + * If entityType is YARN_FLOW_ACTIVITY, entities returned are of type + * {@link FlowActivityEntity}. + * If entityType is YARN_FLOW_RUN, entities returned are of type + * {@link FlowRunEntity}. + * For all other entity types, entities returned are of type + * {@link TimelineEntity}. * @throws IOException */ Set getEntities(String userId, String clusterId, diff --git 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 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 f3f380c..bb16f69 100644 --- 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 +++ 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 @@ -60,6 +60,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix; import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey; import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +import org.apache.hadoop.yarn.webapp.NotFoundException; import com.google.common.base.Preconditions; @@ -197,9 +198,9 @@ protected FlowContext lookupFlowContext(String clusterId, String appId, AppToFlowColumn.FLOW_ID.readResult(result).toString(), ((Number)AppToFlowColumn.FLOW_RUN_ID.readResult(result)).longValue()); } else { - throw new IOException( - "Unable to find the context flow ID and flow run ID for clusterId=" + - clusterId + ", appId=" + appId); + throw new NotFoundException( + "Unable to find the context flow ID and flow run ID for clusterId=" + + clusterId + ", appId=" + appId); } } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java new file mode 100644 index 0000000..791d6ab --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.reader; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.util.List; + +import org.junit.Test; + +public class TestTimelineReaderUtils { + + @Test + public void testSplitUsingEscapeAndDelimChar() throws Exception { + List list = + TimelineReaderUtils.split("*!cluster!*!b**o***!xer!oozie**", '!', '*'); + String[] arr = new String[list.size()]; + arr = list.toArray(arr); + assertArrayEquals(new String[] {"!cluster", "!b*o*!xer", "oozie*"}, arr); + list = TimelineReaderUtils.split("*!cluster!*!b**o***!xer!!", '!', '*'); + arr = new String[list.size()]; + arr = list.toArray(arr); + assertArrayEquals(new String[] {"!cluster", "!b*o*!xer", "", ""}, arr); + } + + @Test + public void testJoinAndEscapeStrings() throws Exception { + assertEquals("*!cluster!*!b**o***!xer!oozie**", + TimelineReaderUtils.joinAndEscapeStrings( + new String[] { "!cluster", "!b*o*!xer", "oozie*"}, '!', '*')); + assertEquals("*!cluster!*!b**o***!xer!!", + TimelineReaderUtils.joinAndEscapeStrings( + new String[] { "!cluster", "!b*o*!xer", "", ""}, '!', '*')); + assertNull(TimelineReaderUtils.joinAndEscapeStrings( + new String[] { "!cluster", "!b*o*!xer", null, ""}, '!', '*')); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java index 9c74e2d..7923eb7 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java @@ -166,7 +166,7 @@ public void testGetEntityDefaultView() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entity/cluster1/app1/app/id_1"); + "timeline/clusters/cluster1/apps/app1/entities/app/id_1"); ClientResponse resp = getResponse(client, uri); TimelineEntity entity = resp.getEntity(TimelineEntity.class); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -189,8 +189,8 @@ public void testGetEntityWithUserAndFlowInfo() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entity/cluster1/app1/app/id_1?userid=user1&" + - "flowname=flow1&flowrunid=1"); + "timeline/clusters/cluster1/apps/app1/entities/app/id_1?" + + "userid=user1&flowname=flow1&flowrunid=1"); ClientResponse resp = getResponse(client, uri); TimelineEntity entity = resp.getEntity(TimelineEntity.class); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -210,7 +210,8 @@ public void testGetEntityCustomFields() throws Exception { try { // Fields are case insensitive. URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entity/cluster1/app1/app/id_1?fields=CONFIGS,Metrics,info"); + "timeline/clusters/cluster1/apps/app1/entities/app/id_1?" + + "fields=CONFIGS,Metrics,info"); ClientResponse resp = getResponse(client, uri); TimelineEntity entity = resp.getEntity(TimelineEntity.class); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -219,7 +220,10 @@ public void testGetEntityCustomFields() throws Exception { assertEquals("app", entity.getType()); assertEquals(3, entity.getConfigs().size()); assertEquals(3, entity.getMetrics().size()); - assertEquals(1, entity.getInfo().size()); + assertTrue("UID should be present", + entity.getInfo().containsKey(TimelineReaderManager.UID_KEY)); + // Includes UID. + assertEquals(2, entity.getInfo().size()); // No events will be returned as events are not part of fields. assertEquals(0, entity.getEvents().size()); } finally { @@ -232,7 +236,8 @@ public void testGetEntityAllFields() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entity/cluster1/app1/app/id_1?fields=ALL"); + "timeline/clusters/cluster1/apps/app1/entities/app/id_1?" + + "fields=ALL"); ClientResponse resp = getResponse(client, uri); TimelineEntity entity = resp.getEntity(TimelineEntity.class); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -241,7 +246,10 @@ public void testGetEntityAllFields() throws Exception { assertEquals("app", entity.getType()); assertEquals(3, entity.getConfigs().size()); assertEquals(3, entity.getMetrics().size()); - assertEquals(1, entity.getInfo().size()); + assertTrue("UID should be present", + entity.getInfo().containsKey(TimelineReaderManager.UID_KEY)); + // Includes UID. + assertEquals(2, entity.getInfo().size()); assertEquals(2, entity.getEvents().size()); } finally { client.destroy(); @@ -253,7 +261,7 @@ public void testGetEntityNotPresent() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entity/cluster1/app1/app/id_10"); + "timeline/clusters/cluster1/apps/app1/entities/app/id_10"); verifyHttpResponse(client, uri, Status.NOT_FOUND); } finally { client.destroy(); @@ -265,7 +273,7 @@ public void testQueryWithoutCluster() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entity/app1/app/id_1"); + "timeline/apps/app1/entities/app/id_1"); ClientResponse resp = getResponse(client, uri); TimelineEntity entity = resp.getEntity(TimelineEntity.class); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -274,7 +282,7 @@ public void testQueryWithoutCluster() throws Exception { assertEquals("app", entity.getType()); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/app1/app"); + "timeline/apps/app1/entities/app"); resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -291,7 +299,7 @@ public void testGetEntities() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app"); + "timeline/clusters/cluster1/apps/app1/entities/app"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -314,7 +322,7 @@ public void testGetEntitiesWithLimit() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?limit=2"); + "timeline/clusters/cluster1/apps/app1/entities/app?limit=2"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -328,7 +336,7 @@ public void testGetEntitiesWithLimit() throws Exception { entities.contains(newEntity("app", "id_4"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entities/cluster1/app1/app?limit=3"); + "clusters/cluster1/apps/app1/entities/app?limit=3"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -346,8 +354,8 @@ public void testGetEntitiesBasedOnCreatedTime() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?createdtimestart=1425016502030&" - + "createdtimeend=1425016502060"); + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "createdtimestart=1425016502030&createdtimeend=1425016502060"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -358,7 +366,8 @@ public void testGetEntitiesBasedOnCreatedTime() throws Exception { entities.contains(newEntity("app", "id_4"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entities/cluster1/app1/app?createdtimeend=1425016502010"); + "clusters/cluster1/apps/app1/entities/app?createdtimeend" + + "=1425016502010"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -368,7 +377,8 @@ public void testGetEntitiesBasedOnCreatedTime() throws Exception { entities.contains(newEntity("app", "id_4"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entities/cluster1/app1/app?createdtimestart=1425016502010"); + "clusters/cluster1/apps/app1/entities/app?createdtimestart=" + + "1425016502010"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -386,8 +396,8 @@ public void testGetEntitiesBasedOnModifiedTime() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?modifiedtimestart=1425016502090" - + "&modifiedtimeend=1425016503020"); + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "modifiedtimestart=1425016502090&modifiedtimeend=1425016503020"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -400,7 +410,8 @@ public void testGetEntitiesBasedOnModifiedTime() throws Exception { entities.contains(newEntity("app", "id_4"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entities/cluster1/app1/app?modifiedtimeend=1425016502090"); + "clusters/cluster1/apps/app1/entities/app?modifiedtimeend=" + + "1425016502090"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -412,7 +423,8 @@ public void testGetEntitiesBasedOnModifiedTime() throws Exception { entities.contains(newEntity("app", "id_3"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entities/cluster1/app1/app?modifiedtimestart=1425016503005"); + "clusters/cluster1/apps/app1/entities/app?modifiedtimestart=" + + "1425016503005"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -430,7 +442,8 @@ public void testGetEntitiesByRelations() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?relatesto=flow:flow1"); + "timeline/clusters/cluster1/apps/app1/entities/app?relatesto=" + + "flow:flow1"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -441,8 +454,8 @@ public void testGetEntitiesByRelations() throws Exception { entities.contains(newEntity("app", "id_1"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entities/cluster1/app1/app?isrelatedto=type1:tid1_2,type2:" + - "tid2_1%60"); + "clusters/cluster1/apps/app1/entities/app?isrelatedto=" + + "type1:tid1_2,type2:tid2_1%60"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -452,8 +465,8 @@ public void testGetEntitiesByRelations() throws Exception { entities.contains(newEntity("app", "id_1"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entities/cluster1/app1/app?isrelatedto=type1:tid1_1:tid1_2" + - ",type2:tid2_1%60"); + "clusters/cluster1/apps/app1/entities/app?isrelatedto=" + + "type1:tid1_1:tid1_2,type2:tid2_1%60"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -471,8 +484,8 @@ public void testGetEntitiesByConfigFilters() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?conffilters=config_1:123," + - "config_3:abc"); + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "conffilters=config_1:123,config_3:abc"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -493,7 +506,8 @@ public void testGetEntitiesByInfoFilters() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?infofilters=info2:3.5"); + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "infofilters=info2:3.5"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -512,7 +526,8 @@ public void testGetEntitiesByMetricFilters() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?metricfilters=metric3"); + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "metricfilters=metric3"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -533,7 +548,8 @@ public void testGetEntitiesByEventFilters() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?eventfilters=event_2,event_4"); + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "eventfilters=event_2,event_4"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -552,10 +568,11 @@ public void testGetEntitiesNoMatch() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?metricfilters=metric7&" + - "isrelatedto=type1:tid1_1;tid1_2,type2:tid2_1%60&relatesto=" + - "flow:flow1&eventfilters=event_2,event_4&infofilters=info2:3.5" + - "&createdtimestart=1425016502030&createdtimeend=1425016502060"); + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "metricfilters=metric7&isrelatedto=type1:tid1_1;tid1_2,type2:tid2_1" + + "%60&relatesto=flow:flow1&eventfilters=event_2,event_4&infofilters=" + + "info2:3.5&createdtimestart=1425016502030&createdtimeend=" + + "1425016502060"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -572,15 +589,15 @@ public void testInvalidValuesHandling() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/app1/app?flowrunid=a23b"); + "timeline/clusters/cluster1/apps/app1/entities/app?flowrunid=a23b"); verifyHttpResponse(client, uri, Status.BAD_REQUEST); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entity/cluster1/app1/app/id_1?flowrunid=2ab15"); + "clusters/cluster1/apps/app1/entities/app/id_1?flowrunid=2ab15"); verifyHttpResponse(client, uri, Status.BAD_REQUEST); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + - "entities/cluster1/app1/app/?limit=#$561av"); + "clusters/cluster1/apps/app1/entities/app?limit=#$561av"); verifyHttpResponse(client, uri, Status.BAD_REQUEST); } finally { client.destroy(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java index 0918d40..9eaa3de 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java @@ -28,8 +28,10 @@ import java.net.URI; import java.net.URL; import java.text.DateFormat; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -349,7 +351,8 @@ public void testGetFlowRun() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowrun/user1/cluster1/flow_name/1002345678919"); + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919"); ClientResponse resp = getResponse(client, uri); FlowRunEntity entity = resp.getEntity(FlowRunEntity.class); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -366,7 +369,7 @@ public void testGetFlowRun() throws Exception { // Query without specifying cluster ID. uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowrun/user1/flow_name/1002345678919"); + "timeline/users/user1/flows/flow_name/runs/1002345678919"); resp = getResponse(client, uri); entity = resp.getEntity(FlowRunEntity.class); assertNotNull(entity); @@ -390,7 +393,7 @@ public void testGetFlowRuns() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowruns/user1/cluster1/flow_name"); + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -408,8 +411,8 @@ public void testGetFlowRuns() throws Exception { assertEquals(0, entity.getMetrics().size()); } - uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowruns/user1/cluster1/flow_name?limit=1"); + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "clusters/cluster1/users/user1/flows/flow_name/runs?limit=1"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -424,7 +427,7 @@ public void testGetFlowRuns() throws Exception { } uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowruns/user1/cluster1/flow_name?" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + "createdtimestart=1425016501030"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); @@ -440,7 +443,7 @@ public void testGetFlowRuns() throws Exception { } uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowruns/user1/cluster1/flow_name?" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + "createdtimestart=1425016500999&createdtimeend=1425016501035"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); @@ -459,7 +462,7 @@ public void testGetFlowRuns() throws Exception { } uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowruns/user1/cluster1/flow_name?" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + "createdtimeend=1425016501030"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); @@ -475,7 +478,8 @@ public void testGetFlowRuns() throws Exception { } uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowruns/user1/cluster1/flow_name?fields=metrics"); + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + + "fields=metrics"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); @@ -498,11 +502,262 @@ public void testGetFlowRuns() throws Exception { } @Test + public void testGetEntitiesByUID() throws Exception { + Client client = createClient(); + try { + // Query all flows. + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/flows"); + ClientResponse resp = getResponse(client, uri); + Set flowEntities = + resp.getEntity(new GenericType>(){}); + assertNotNull(flowEntities); + assertEquals(2, flowEntities.size()); + List listFlowUIDs = new ArrayList(); + for (FlowActivityEntity entity : flowEntities) { + String flowUID = + (String)entity.getInfo().get(TimelineReaderManager.UID_KEY); + listFlowUIDs.add(flowUID); + assertEquals(TimelineUIDConverter.FLOW_UID.encodeUID( + new TimelineReaderContext(entity.getCluster(), entity.getUser(), + entity.getFlowName(), null, null, null, null)), flowUID); + assertTrue((entity.getId().endsWith("@flow_name") && + entity.getFlowRuns().size() == 2) || + (entity.getId().endsWith("@flow_name2") && + entity.getFlowRuns().size() == 1)); + } + + // Query flowruns based on UID returned in query above. + List listFlowRunUIDs = new ArrayList(); + for (String flowUID : listFlowUIDs) { + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/flow-uid/" + flowUID + "/runs"); + resp = getResponse(client, uri); + Set frEntities = + resp.getEntity(new GenericType>(){}); + assertNotNull(frEntities); + for (FlowRunEntity entity : frEntities) { + String flowRunUID = + (String)entity.getInfo().get(TimelineReaderManager.UID_KEY); + listFlowRunUIDs.add(flowRunUID); + assertEquals(TimelineUIDConverter.FLOWRUN_UID.encodeUID( + new TimelineReaderContext("cluster1", entity.getUser(), + entity.getName(), entity.getRunId(), null, null, null)), + flowRunUID); + } + } + assertEquals(3, listFlowRunUIDs.size()); + + // Query single flowrun based on UIDs' returned in query to get flowruns. + for (String flowRunUID : listFlowRunUIDs) { + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/run-uid/" + flowRunUID); + resp = getResponse(client, uri); + FlowRunEntity entity = resp.getEntity(FlowRunEntity.class); + assertNotNull(entity); + } + + // Query apps based on UIDs' returned in query to get flowruns. + List listAppUIDs = new ArrayList(); + for (String flowRunUID : listFlowRunUIDs) { + TimelineReaderContext context = + TimelineUIDConverter.FLOWRUN_UID.decodeUID(flowRunUID); + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/run-uid/" + flowRunUID + "/apps"); + resp = getResponse(client, uri); + Set appEntities = + resp.getEntity(new GenericType>(){}); + assertNotNull(appEntities); + for (TimelineEntity entity : appEntities) { + String appUID = + (String)entity.getInfo().get(TimelineReaderManager.UID_KEY); + listAppUIDs.add(appUID); + assertEquals(TimelineUIDConverter.APPLICATION_UID.encodeUID( + new TimelineReaderContext(context.getClusterId(), + context.getUserId(), context.getFlowName(), + context.getFlowRunId(), entity.getId(), null, null)), appUID); + } + } + assertEquals(4, listAppUIDs.size()); + + // Query single app based on UIDs' returned in query to get apps. + for (String appUID : listAppUIDs) { + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/app-uid/" + appUID); + resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + } + + // Query entities based on UIDs' returned in query to get apps and + // a specific entity type(in this case type1). + List listEntityUIDs = new ArrayList(); + for (String appUID : listAppUIDs) { + TimelineReaderContext context = + TimelineUIDConverter.APPLICATION_UID.decodeUID(appUID); + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/app-uid/" + appUID + "/entities/type1"); + resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + for (TimelineEntity entity : entities) { + String entityUID = + (String)entity.getInfo().get(TimelineReaderManager.UID_KEY); + listEntityUIDs.add(entityUID); + assertEquals(TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID( + new TimelineReaderContext(context.getClusterId(), + context.getUserId(), context.getFlowName(), + context.getFlowRunId(), context.getAppId(), "type1", + entity.getId())), entityUID); + } + } + assertEquals(2, listEntityUIDs.size()); + + // Query single entity based on UIDs' returned in query to get entities. + for (String entityUID : listEntityUIDs) { + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/entity-uid/" + entityUID); + resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + } + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/flow-uid/dummy:flow/runs"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/run-uid/dummy:flowrun"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + // Run Id is not a numerical value. + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/run-uid/some:dummy:flow:123v456"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/run-uid/dummy:flowrun/apps"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/app-uid/dummy:app"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/app-uid/dummy:app/entities/type1"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/entity-uid/dummy:entity"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } + + @Test + public void testUIDQueryWithAndWithoutFlowContextInfo() throws Exception { + Client client = createClient(); + try { + String appUIDWithFlowInfo = + "cluster1!user1!flow_name!1002345678919!application_1111111111_1111"; + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"+ + "timeline/app-uid/" + appUIDWithFlowInfo); + ClientResponse resp = getResponse(client, uri); + TimelineEntity appEntity1 = resp.getEntity(TimelineEntity.class); + assertNotNull(appEntity1); + assertEquals( + TimelineEntityType.YARN_APPLICATION.toString(), appEntity1.getType()); + assertEquals("application_1111111111_1111", appEntity1.getId()); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "app-uid/" + appUIDWithFlowInfo + "/entities/type1"); + resp = getResponse(client, uri); + Set entities1 = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities1); + assertEquals(2, entities1.size()); + for (TimelineEntity entity : entities1) { + assertNotNull(entity.getInfo()); + assertEquals(1, entity.getInfo().size()); + String uid = + (String) entity.getInfo().get(TimelineReaderManager.UID_KEY); + assertNotNull(uid); + assertTrue(uid.equals(appUIDWithFlowInfo + "!type1!entity1") || + uid.equals(appUIDWithFlowInfo + "!type1!entity2")); + } + + String appUIDWithoutFlowInfo = "cluster1!application_1111111111_1111"; + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+ + "app-uid/" + appUIDWithoutFlowInfo); + resp = getResponse(client, uri);; + TimelineEntity appEntity2 = resp.getEntity(TimelineEntity.class); + assertNotNull(appEntity2); + assertEquals( + TimelineEntityType.YARN_APPLICATION.toString(), appEntity2.getType()); + assertEquals("application_1111111111_1111", appEntity2.getId()); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "app-uid/" + appUIDWithoutFlowInfo + "/entities/type1"); + resp = getResponse(client, uri); + Set entities2 = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities2); + assertEquals(2, entities2.size()); + for (TimelineEntity entity : entities2) { + assertNotNull(entity.getInfo()); + assertEquals(1, entity.getInfo().size()); + String uid = + (String) entity.getInfo().get(TimelineReaderManager.UID_KEY); + assertNotNull(uid); + assertTrue(uid.equals(appUIDWithoutFlowInfo + "!type1!entity1") || + uid.equals(appUIDWithoutFlowInfo + "!type1!entity2")); + } + + String entityUIDWithFlowInfo = appUIDWithFlowInfo + "!type1!entity1"; + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+ + "entity-uid/" + entityUIDWithFlowInfo); + resp = getResponse(client, uri);; + TimelineEntity singleEntity1 = resp.getEntity(TimelineEntity.class); + assertNotNull(singleEntity1); + assertEquals("type1", singleEntity1.getType()); + assertEquals("entity1", singleEntity1.getId()); + + String entityUIDWithoutFlowInfo = + appUIDWithoutFlowInfo + "!type1!entity1"; + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+ + "entity-uid/" + entityUIDWithoutFlowInfo); + resp = getResponse(client, uri);; + TimelineEntity singleEntity2 = resp.getEntity(TimelineEntity.class); + assertNotNull(singleEntity2); + assertEquals("type1", singleEntity2.getType()); + assertEquals("entity1", singleEntity2.getId()); + } finally { + client.destroy(); + } + } + + @Test + public void testUIDNotProperlyEscaped() throws Exception { + Client client = createClient(); + try { + String appUID = + "cluster1!user*1!flow_name!1002345678919!application_1111111111_1111"; + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"+ + "timeline/app-uid/" + appUID); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } + + @Test public void testGetFlows() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1"); + "timeline/clusters/cluster1/flows"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -524,7 +779,7 @@ public void testGetFlows() throws Exception { assertEquals(2, entities.size()); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?limit=1"); + "timeline/clusters/cluster1/flows?limit=1"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertNotNull(entities); @@ -532,8 +787,8 @@ public void testGetFlows() throws Exception { DateFormat fmt = TimelineReaderWebServices.DATE_FORMAT.get(); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?daterange=" + fmt.format(dayTs) + "-" + - fmt.format(dayTs + (2*86400000L))); + "timeline/clusters/cluster1/flows?daterange=" + fmt.format(dayTs) + + "-" + fmt.format(dayTs + (2*86400000L))); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertNotNull(entities); @@ -546,7 +801,7 @@ public void testGetFlows() throws Exception { } uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?daterange=" + + "timeline/clusters/cluster1/flows?daterange=" + fmt.format(dayTs + (4*86400000L))); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); @@ -554,7 +809,7 @@ public void testGetFlows() throws Exception { assertEquals(0, entities.size()); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?daterange=-" + + "timeline/clusters/cluster1/flows?daterange=-" + fmt.format(dayTs + (2*86400000L))); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); @@ -562,7 +817,7 @@ public void testGetFlows() throws Exception { assertEquals(2, entities.size()); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?daterange=" + + "timeline/clusters/cluster1/flows?daterange=" + fmt.format(dayTs - (2*86400000L)) + "-"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); @@ -570,19 +825,19 @@ public void testGetFlows() throws Exception { assertEquals(2, entities.size()); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?daterange=20150711:20150714"); + "timeline/clusters/cluster1/flows?daterange=20150711:20150714"); verifyHttpResponse(client, uri, Status.BAD_REQUEST); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?daterange=20150714-20150711"); + "timeline/clusters/cluster1/flows?daterange=20150714-20150711"); verifyHttpResponse(client, uri, Status.BAD_REQUEST); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?daterange=2015071129-20150712"); + "timeline/clusters/cluster1/flows?daterange=2015071129-20150712"); verifyHttpResponse(client, uri, Status.BAD_REQUEST); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster1?daterange=20150711-2015071243"); + "timeline/clusters/cluster1/flows?daterange=20150711-2015071243"); verifyHttpResponse(client, uri, Status.BAD_REQUEST); } finally { client.destroy(); @@ -594,7 +849,7 @@ public void testGetApp() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/app/cluster1/application_1111111111_1111?" + + "timeline/clusters/cluster1/apps/application_1111111111_1111?" + "userid=user1&fields=ALL&flowname=flow_name&flowrunid=1002345678919"); ClientResponse resp = getResponse(client, uri); TimelineEntity entity = resp.getEntity(TimelineEntity.class); @@ -612,7 +867,7 @@ public void testGetApp() throws Exception { } uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/app/application_1111111111_2222?userid=user1" + + "timeline/apps/application_1111111111_2222?userid=user1" + "&fields=metrics&flowname=flow_name&flowrunid=1002345678919"); resp = getResponse(client, uri); entity = resp.getEntity(TimelineEntity.class); @@ -635,7 +890,7 @@ public void testGetAppWithoutFlowInfo() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/app/cluster1/application_1111111111_1111?" + + "timeline/clusters/cluster1/apps/application_1111111111_1111?" + "fields=ALL"); ClientResponse resp = getResponse(client, uri); TimelineEntity entity = resp.getEntity(TimelineEntity.class); @@ -661,7 +916,8 @@ public void testGetEntityWithoutFlowInfo() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entity/cluster1/application_1111111111_1111/type1/entity1"); + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity1"); ClientResponse resp = getResponse(client, uri); TimelineEntity entity = resp.getEntity(TimelineEntity.class); assertNotNull(entity); @@ -677,7 +933,8 @@ public void testGetEntitiesWithoutFlowInfo() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/entities/cluster1/application_1111111111_1111/type1"); + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -697,8 +954,8 @@ public void testGetFlowRunApps() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowrunapps/user1/cluster1/flow_name/1002345678919?" + - "fields=ALL"); + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919/apps?fields=ALL"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -714,14 +971,15 @@ public void testGetFlowRunApps() throws Exception { // Query without specifying cluster ID. uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowrunapps/user1/flow_name/1002345678919"); + "timeline/users/user1/flows/flow_name/runs/1002345678919/apps"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertNotNull(entities); assertEquals(2, entities.size()); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowrunapps/user1/flow_name/1002345678919?limit=1"); + "timeline/users/user1/flows/flow_name/runs/1002345678919/" + + "apps?limit=1"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertNotNull(entities); @@ -736,7 +994,8 @@ public void testGetFlowApps() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowapps/user1/cluster1/flow_name?fields=ALL"); + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "fields=ALL"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -754,14 +1013,14 @@ public void testGetFlowApps() throws Exception { // Query without specifying cluster ID. uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowapps/user1/flow_name"); + "timeline/users/user1/flows/flow_name/apps"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertNotNull(entities); assertEquals(3, entities.size()); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowapps/user1/flow_name?limit=1"); + "timeline/users/user1/flows/flow_name/apps?limit=1"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertNotNull(entities); @@ -777,8 +1036,8 @@ public void testGetFlowAppsFilters() throws Exception { try { String entityType = TimelineEntityType.YARN_APPLICATION.toString(); URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowapps/user1/cluster1/flow_name?eventfilters=" + - ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "eventfilters=" + ApplicationMetricsConstants.FINISHED_EVENT_TYPE); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -788,8 +1047,8 @@ public void testGetFlowAppsFilters() throws Exception { newEntity(entityType, "application_1111111111_1111"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowapps/user1/cluster1/flow_name?metricfilters=" + - "HDFS_BYTES_READ"); + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "metricfilters=HDFS_BYTES_READ"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertNotNull(entities); @@ -798,8 +1057,8 @@ public void testGetFlowAppsFilters() throws Exception { newEntity(entityType, "application_1111111111_1111"))); uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowapps/user1/cluster1/flow_name?conffilters=" + - "cfg1:value1"); + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "conffilters=cfg1:value1"); resp = getResponse(client, uri); entities = resp.getEntity(new GenericType>(){}); assertNotNull(entities); @@ -816,7 +1075,8 @@ public void testGetFlowRunNotPresent() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowrun/user1/cluster1/flow_name/1002345678929"); + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678929"); verifyHttpResponse(client, uri, Status.NOT_FOUND); } finally { client.destroy(); @@ -828,7 +1088,7 @@ public void testGetFlowsNotPresent() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flows/cluster2"); + "timeline/clusters/cluster2/flows"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -845,8 +1105,7 @@ public void testGetAppNotPresent() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/app/user1/cluster1/flow_name/1002345678919/" + - "application_1111111111_1378"); + "timeline/clusters/cluster1/apps/application_1111111111_1378"); verifyHttpResponse(client, uri, Status.NOT_FOUND); } finally { client.destroy(); @@ -858,7 +1117,8 @@ public void testGetFlowRunAppsNotPresent() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowrunapps/user1/cluster2/flow_name/1002345678919"); + "timeline/clusters/cluster2/users/user1/flows/flow_name/runs/" + + "1002345678919/apps"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); @@ -875,7 +1135,7 @@ public void testGetFlowAppsNotPresent() throws Exception { Client client = createClient(); try { URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + - "timeline/flowapps/user1/cluster2/flow_name55"); + "timeline/clusters/cluster2/users/user1/flows/flow_name55/apps"); ClientResponse resp = getResponse(client, uri); Set entities = resp.getEntity(new GenericType>(){}); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java new file mode 100644 index 0000000..8d6235d --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java @@ -0,0 +1,97 @@ +/** + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +import org.junit.Test; + +public class TestTimelineUIDConverter { + + @Test + public void testUIDEncodingDecoding() throws Exception { + TimelineReaderContext context = new TimelineReaderContext( + "!cluster", "!b*o*!xer", "oozie*", null, null, null, null); + String uid = TimelineUIDConverter.FLOW_UID.encodeUID(context); + assertEquals("*!cluster!*!b**o***!xer!oozie**", uid); + assertEquals(context, TimelineUIDConverter.FLOW_UID.decodeUID(uid)); + + context = new TimelineReaderContext("!cluster*", "!b*o!!x!*er", "*oozie!", + 123L, null, null, null); + uid = TimelineUIDConverter.FLOWRUN_UID.encodeUID(context); + assertEquals("*!cluster**!*!b**o*!*!x*!**er!**oozie*!!123", uid); + assertEquals(context, TimelineUIDConverter.FLOWRUN_UID.decodeUID(uid)); + + context = new TimelineReaderContext("yarn_cluster", "root", "hive_join", + 1234L, "application_1111111111_1111", null, null); + uid = TimelineUIDConverter.APPLICATION_UID.encodeUID(context); + assertEquals( + "yarn_cluster!root!hive_join!1234!application_1111111111_1111", uid); + assertEquals(context, TimelineUIDConverter.APPLICATION_UID.decodeUID(uid)); + context = new TimelineReaderContext("yarn_cluster", null, null, null, + "application_1111111111_1111", null, null); + uid = TimelineUIDConverter.APPLICATION_UID.encodeUID(context); + assertEquals("yarn_cluster!application_1111111111_1111", uid); + assertEquals(context, TimelineUIDConverter.APPLICATION_UID.decodeUID(uid)); + + context = new TimelineReaderContext("yarn_cluster", "root", "hive_join", + 1234L, "application_1111111111_1111", "YARN_CONTAINER", + "container_1111111111_1111_01_000001"); + uid = TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context); + assertEquals("yarn_cluster!root!hive_join!1234!application_1111111111_1111!" + + "YARN_CONTAINER!container_1111111111_1111_01_000001", uid); + assertEquals( + context, TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uid)); + context = new TimelineReaderContext("yarn_cluster",null, null, null, + "application_1111111111_1111", "YARN_CONTAINER", + "container_1111111111_1111_01_000001"); + uid = TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context); + assertEquals("yarn_cluster!application_1111111111_1111!YARN_CONTAINER!" + + "container_1111111111_1111_01_000001", uid); + assertEquals( + context, TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uid)); + } + + @Test + public void testUIDNotProperlyEscaped() throws Exception { + try { + TimelineUIDConverter.FLOW_UID.decodeUID("*!cluster!*!b*o***!xer!oozie**"); + fail("UID not properly escaped. Exception should have been thrown."); + } catch (IllegalArgumentException e) { + } + + try { + TimelineUIDConverter.FLOW_UID.decodeUID("*!cluster!*!b**o***!xer!oozie*"); + fail("UID not properly escaped. Exception should have been thrown."); + } catch (IllegalArgumentException e) { + } + + try { + TimelineUIDConverter.FLOW_UID.decodeUID("*!cluster!*!b**o***xer!oozie*"); + fail("UID not properly escaped. Exception should have been thrown."); + } catch (IllegalArgumentException e) { + } + assertNull(TimelineUIDConverter. + FLOW_UID.decodeUID("!cluster!*!b**o***!xer!oozie**")); + assertNull(TimelineUIDConverter. + FLOW_UID.decodeUID("*!cluster!*!b**o**!xer!oozie**")); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java index de66a17..4d8d588 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java @@ -128,7 +128,7 @@ private void testAggregator(OfflineAggregationInfo aggregationInfo) TimelineEntities te = new TimelineEntities(); te.addEntity(getTestAggregationTimelineEntity()); TimelineCollectorContext context = new TimelineCollectorContext("cluster_1", - "user1", "testFlow", null, 0, null); + "user1", "testFlow", null, 0L, null); storage.writeAggregatedEntity(context, te, aggregationInfo);