YARN-6861. Reader API for sub application entities (Rohith Sharma K S via Varun Saxena)
This commit is contained in:
parent
7fd6ae2479
commit
b2efebdd07
|
@ -71,6 +71,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
|
|||
private static long ts = System.currentTimeMillis();
|
||||
private static long dayTs =
|
||||
HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts);
|
||||
private static String doAsUser = "remoteuser";
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBeforeClass() throws Exception {
|
||||
|
@ -337,7 +338,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
|
|||
HBaseTimelineWriterImpl hbi = null;
|
||||
Configuration c1 = getHBaseTestingUtility().getConfiguration();
|
||||
UserGroupInformation remoteUser =
|
||||
UserGroupInformation.createRemoteUser(user);
|
||||
UserGroupInformation.createRemoteUser(doAsUser);
|
||||
try {
|
||||
hbi = new HBaseTimelineWriterImpl();
|
||||
hbi.init(c1);
|
||||
|
@ -2263,60 +2264,69 @@ public class TestTimelineReaderWebServicesHBaseStorage
|
|||
public void testGenericEntitiesForPagination() throws Exception {
|
||||
Client client = createClient();
|
||||
try {
|
||||
int limit = 10;
|
||||
String queryParam = "?limit=" + limit;
|
||||
String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/"
|
||||
+ "timeline/clusters/cluster1/apps/application_1111111111_1111/"
|
||||
+ "entities/entitytype";
|
||||
URI uri = URI.create(resourceUri + queryParam);
|
||||
|
||||
ClientResponse resp = getResponse(client, uri);
|
||||
List<TimelineEntity> entities =
|
||||
resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// verify for entity-10 to entity-1 in descending order.
|
||||
verifyPaginatedEntites(entities, limit, limit);
|
||||
|
||||
limit = 4;
|
||||
queryParam = "?limit=" + limit;
|
||||
uri = URI.create(resourceUri + queryParam);
|
||||
resp = getResponse(client, uri);
|
||||
entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// verify for entity-10 to entity-7 in descending order.
|
||||
TimelineEntity entity = verifyPaginatedEntites(entities, limit, 10);
|
||||
|
||||
queryParam = "?limit=" + limit + "&fromid="
|
||||
+ entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
|
||||
uri = URI.create(resourceUri + queryParam);
|
||||
resp = getResponse(client, uri);
|
||||
entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// verify for entity-7 to entity-4 in descending order.
|
||||
entity = verifyPaginatedEntites(entities, limit, 7);
|
||||
|
||||
queryParam = "?limit=" + limit + "&fromid="
|
||||
+ entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
|
||||
uri = URI.create(resourceUri + queryParam);
|
||||
resp = getResponse(client, uri);
|
||||
entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// verify for entity-4 to entity-1 in descending order.
|
||||
entity = verifyPaginatedEntites(entities, limit, 4);
|
||||
|
||||
queryParam = "?limit=" + limit + "&fromid="
|
||||
+ entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
|
||||
uri = URI.create(resourceUri + queryParam);
|
||||
resp = getResponse(client, uri);
|
||||
entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// always entity-1 will be retrieved
|
||||
entity = verifyPaginatedEntites(entities, 1, 1);
|
||||
verifyEntitiesForPagination(client, resourceUri);
|
||||
resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/"
|
||||
+ "timeline/clusters/cluster1/users/" + doAsUser
|
||||
+ "/entities/entitytype";
|
||||
verifyEntitiesForPagination(client, resourceUri);
|
||||
} finally {
|
||||
client.destroy();
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyEntitiesForPagination(Client client, String resourceUri)
|
||||
throws Exception {
|
||||
int limit = 10;
|
||||
String queryParam = "?limit=" + limit;
|
||||
URI uri = URI.create(resourceUri + queryParam);
|
||||
|
||||
ClientResponse resp = getResponse(client, uri);
|
||||
List<TimelineEntity> entities =
|
||||
resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// verify for entity-10 to entity-1 in descending order.
|
||||
verifyPaginatedEntites(entities, limit, limit);
|
||||
|
||||
limit = 4;
|
||||
queryParam = "?limit=" + limit;
|
||||
uri = URI.create(resourceUri + queryParam);
|
||||
resp = getResponse(client, uri);
|
||||
entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// verify for entity-10 to entity-7 in descending order.
|
||||
TimelineEntity entity = verifyPaginatedEntites(entities, limit, 10);
|
||||
|
||||
queryParam = "?limit=" + limit + "&fromid="
|
||||
+ entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
|
||||
uri = URI.create(resourceUri + queryParam);
|
||||
resp = getResponse(client, uri);
|
||||
entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// verify for entity-7 to entity-4 in descending order.
|
||||
entity = verifyPaginatedEntites(entities, limit, 7);
|
||||
|
||||
queryParam = "?limit=" + limit + "&fromid="
|
||||
+ entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
|
||||
uri = URI.create(resourceUri + queryParam);
|
||||
resp = getResponse(client, uri);
|
||||
entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// verify for entity-4 to entity-1 in descending order.
|
||||
entity = verifyPaginatedEntites(entities, limit, 4);
|
||||
|
||||
queryParam = "?limit=" + limit + "&fromid="
|
||||
+ entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
|
||||
uri = URI.create(resourceUri + queryParam);
|
||||
resp = getResponse(client, uri);
|
||||
entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
|
||||
});
|
||||
// always entity-1 will be retrieved
|
||||
entity = verifyPaginatedEntites(entities, 1, 1);
|
||||
}
|
||||
|
||||
private TimelineEntity verifyPaginatedEntites(List<TimelineEntity> entities,
|
||||
int limit, int startFrom) {
|
||||
assertNotNull(entities);
|
||||
|
|
|
@ -135,7 +135,7 @@ class GenericEntityReader extends TimelineEntityReader {
|
|||
*
|
||||
* @return true if we need to fetch some of the columns, false otherwise.
|
||||
*/
|
||||
private boolean fetchPartialEventCols(TimelineFilterList eventFilters,
|
||||
protected boolean fetchPartialEventCols(TimelineFilterList eventFilters,
|
||||
EnumSet<Field> fieldsToRetrieve) {
|
||||
return (eventFilters != null && !eventFilters.getFilterList().isEmpty() &&
|
||||
!hasField(fieldsToRetrieve, Field.EVENTS));
|
||||
|
@ -146,7 +146,7 @@ class GenericEntityReader extends TimelineEntityReader {
|
|||
*
|
||||
* @return true if we need to fetch some of the columns, false otherwise.
|
||||
*/
|
||||
private boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
|
||||
protected boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
|
||||
EnumSet<Field> fieldsToRetrieve) {
|
||||
return (relatesTo != null && !relatesTo.getFilterList().isEmpty() &&
|
||||
!hasField(fieldsToRetrieve, Field.RELATES_TO));
|
||||
|
|
|
@ -0,0 +1,488 @@
|
|||
/**
|
||||
* 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.storage.reader;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Query;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.filter.FamilyFilter;
|
||||
import org.apache.hadoop.hbase.filter.FilterList;
|
||||
import org.apache.hadoop.hbase.filter.FilterList.Operator;
|
||||
import org.apache.hadoop.hbase.filter.PageFilter;
|
||||
import org.apache.hadoop.hbase.filter.QualifierFilter;
|
||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnFamily;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKeyPrefix;
|
||||
import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable;
|
||||
import org.apache.hadoop.yarn.webapp.BadRequestException;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
class SubApplicationEntityReader extends GenericEntityReader {
|
||||
private static final SubApplicationTable SUB_APPLICATION_TABLE =
|
||||
new SubApplicationTable();
|
||||
|
||||
SubApplicationEntityReader(TimelineReaderContext ctxt,
|
||||
TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
|
||||
super(ctxt, entityFilters, toRetrieve);
|
||||
}
|
||||
|
||||
SubApplicationEntityReader(TimelineReaderContext ctxt,
|
||||
TimelineDataToRetrieve toRetrieve) {
|
||||
super(ctxt, toRetrieve);
|
||||
}
|
||||
|
||||
/**
|
||||
* Uses the {@link SubApplicationTable}.
|
||||
*/
|
||||
protected BaseTable<?> getTable() {
|
||||
return SUB_APPLICATION_TABLE;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FilterList constructFilterListBasedOnFilters() throws IOException {
|
||||
// Filters here cannot be null for multiple entity reads as they are set in
|
||||
// augmentParams if null.
|
||||
FilterList listBasedOnFilters = new FilterList();
|
||||
TimelineEntityFilters filters = getFilters();
|
||||
// Create filter list based on created time range and add it to
|
||||
// listBasedOnFilters.
|
||||
long createdTimeBegin = filters.getCreatedTimeBegin();
|
||||
long createdTimeEnd = filters.getCreatedTimeEnd();
|
||||
if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
|
||||
listBasedOnFilters.addFilter(TimelineFilterUtils
|
||||
.createSingleColValueFiltersByRange(SubApplicationColumn.CREATED_TIME,
|
||||
createdTimeBegin, createdTimeEnd));
|
||||
}
|
||||
// Create filter list based on metric filters and add it to
|
||||
// listBasedOnFilters.
|
||||
TimelineFilterList metricFilters = filters.getMetricFilters();
|
||||
if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) {
|
||||
listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
|
||||
SubApplicationColumnPrefix.METRIC, metricFilters));
|
||||
}
|
||||
// Create filter list based on config filters and add it to
|
||||
// listBasedOnFilters.
|
||||
TimelineFilterList configFilters = filters.getConfigFilters();
|
||||
if (configFilters != null && !configFilters.getFilterList().isEmpty()) {
|
||||
listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
|
||||
SubApplicationColumnPrefix.CONFIG, configFilters));
|
||||
}
|
||||
// Create filter list based on info filters and add it to listBasedOnFilters
|
||||
TimelineFilterList infoFilters = filters.getInfoFilters();
|
||||
if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) {
|
||||
listBasedOnFilters.addFilter(TimelineFilterUtils
|
||||
.createHBaseFilterList(SubApplicationColumnPrefix.INFO, infoFilters));
|
||||
}
|
||||
return listBasedOnFilters;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add {@link QualifierFilter} filters to filter list for each column of
|
||||
* entity table.
|
||||
*
|
||||
* @param list filter list to which qualifier filters have to be added.
|
||||
*/
|
||||
protected void updateFixedColumns(FilterList list) {
|
||||
for (SubApplicationColumn column : SubApplicationColumn.values()) {
|
||||
list.addFilter(new QualifierFilter(CompareOp.EQUAL,
|
||||
new BinaryComparator(column.getColumnQualifierBytes())));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a filter list which indicates that only some of the column
|
||||
* qualifiers in the info column family will be returned in result.
|
||||
*
|
||||
* @param isApplication If true, it means operations are to be performed for
|
||||
* application table, otherwise for entity table.
|
||||
* @return filter list.
|
||||
* @throws IOException if any problem occurs while creating filter list.
|
||||
*/
|
||||
private FilterList createFilterListForColsOfInfoFamily() throws IOException {
|
||||
FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE);
|
||||
// Add filters for each column in entity table.
|
||||
updateFixedColumns(infoFamilyColsFilter);
|
||||
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
|
||||
// If INFO field has to be retrieved, add a filter for fetching columns
|
||||
// with INFO column prefix.
|
||||
if (hasField(fieldsToRetrieve, Field.INFO)) {
|
||||
infoFamilyColsFilter.addFilter(
|
||||
TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL,
|
||||
SubApplicationColumnPrefix.INFO));
|
||||
}
|
||||
TimelineFilterList relatesTo = getFilters().getRelatesTo();
|
||||
if (hasField(fieldsToRetrieve, Field.RELATES_TO)) {
|
||||
// If RELATES_TO field has to be retrieved, add a filter for fetching
|
||||
// columns with RELATES_TO column prefix.
|
||||
infoFamilyColsFilter.addFilter(
|
||||
TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL,
|
||||
SubApplicationColumnPrefix.RELATES_TO));
|
||||
} else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) {
|
||||
// Even if fields to retrieve does not contain RELATES_TO, we still
|
||||
// need to have a filter to fetch some of the column qualifiers if
|
||||
// relatesTo filters are specified. relatesTo filters will then be
|
||||
// matched after fetching rows from HBase.
|
||||
Set<String> relatesToCols =
|
||||
TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo);
|
||||
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
|
||||
SubApplicationColumnPrefix.RELATES_TO, relatesToCols));
|
||||
}
|
||||
TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo();
|
||||
if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
|
||||
// If IS_RELATED_TO field has to be retrieved, add a filter for fetching
|
||||
// columns with IS_RELATED_TO column prefix.
|
||||
infoFamilyColsFilter.addFilter(
|
||||
TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL,
|
||||
SubApplicationColumnPrefix.IS_RELATED_TO));
|
||||
} else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) {
|
||||
// Even if fields to retrieve does not contain IS_RELATED_TO, we still
|
||||
// need to have a filter to fetch some of the column qualifiers if
|
||||
// isRelatedTo filters are specified. isRelatedTo filters will then be
|
||||
// matched after fetching rows from HBase.
|
||||
Set<String> isRelatedToCols =
|
||||
TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo);
|
||||
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
|
||||
SubApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols));
|
||||
}
|
||||
TimelineFilterList eventFilters = getFilters().getEventFilters();
|
||||
if (hasField(fieldsToRetrieve, Field.EVENTS)) {
|
||||
// If EVENTS field has to be retrieved, add a filter for fetching columns
|
||||
// with EVENT column prefix.
|
||||
infoFamilyColsFilter.addFilter(
|
||||
TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL,
|
||||
SubApplicationColumnPrefix.EVENT));
|
||||
} else if (eventFilters != null
|
||||
&& !eventFilters.getFilterList().isEmpty()) {
|
||||
// Even if fields to retrieve does not contain EVENTS, we still need to
|
||||
// have a filter to fetch some of the column qualifiers on the basis of
|
||||
// event filters specified. Event filters will then be matched after
|
||||
// fetching rows from HBase.
|
||||
Set<String> eventCols =
|
||||
TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters);
|
||||
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
|
||||
SubApplicationColumnPrefix.EVENT, eventCols));
|
||||
}
|
||||
return infoFamilyColsFilter;
|
||||
}
|
||||
|
||||
/**
|
||||
* Exclude column prefixes via filters which are not required(based on fields
|
||||
* to retrieve) from info column family. These filters are added to filter
|
||||
* list which contains a filter for getting info column family.
|
||||
*
|
||||
* @param infoColFamilyList filter list for info column family.
|
||||
*/
|
||||
private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) {
|
||||
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
|
||||
// Events not required.
|
||||
if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
|
||||
infoColFamilyList.addFilter(
|
||||
TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
|
||||
SubApplicationColumnPrefix.EVENT));
|
||||
}
|
||||
// info not required.
|
||||
if (!hasField(fieldsToRetrieve, Field.INFO)) {
|
||||
infoColFamilyList.addFilter(
|
||||
TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
|
||||
SubApplicationColumnPrefix.INFO));
|
||||
}
|
||||
// is related to not required.
|
||||
if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
|
||||
infoColFamilyList.addFilter(
|
||||
TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
|
||||
SubApplicationColumnPrefix.IS_RELATED_TO));
|
||||
}
|
||||
// relates to not required.
|
||||
if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
|
||||
infoColFamilyList.addFilter(
|
||||
TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
|
||||
SubApplicationColumnPrefix.RELATES_TO));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates filter list based on fields for confs and metrics to retrieve.
|
||||
*
|
||||
* @param listBasedOnFields filter list based on fields.
|
||||
* @throws IOException if any problem occurs while updating filter list.
|
||||
*/
|
||||
private void updateFilterForConfsAndMetricsToRetrieve(
|
||||
FilterList listBasedOnFields) throws IOException {
|
||||
TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
|
||||
// Please note that if confsToRetrieve is specified, we would have added
|
||||
// CONFS to fields to retrieve in augmentParams() even if not specified.
|
||||
if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) {
|
||||
// Create a filter list for configs.
|
||||
listBasedOnFields.addFilter(
|
||||
TimelineFilterUtils.createFilterForConfsOrMetricsToRetrieve(
|
||||
dataToRetrieve.getConfsToRetrieve(),
|
||||
SubApplicationColumnFamily.CONFIGS,
|
||||
SubApplicationColumnPrefix.CONFIG));
|
||||
}
|
||||
|
||||
// Please note that if metricsToRetrieve is specified, we would have added
|
||||
// METRICS to fields to retrieve in augmentParams() even if not specified.
|
||||
if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) {
|
||||
// Create a filter list for metrics.
|
||||
listBasedOnFields.addFilter(
|
||||
TimelineFilterUtils.createFilterForConfsOrMetricsToRetrieve(
|
||||
dataToRetrieve.getMetricsToRetrieve(),
|
||||
SubApplicationColumnFamily.METRICS,
|
||||
SubApplicationColumnPrefix.METRIC));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FilterList constructFilterListBasedOnFields() throws IOException {
|
||||
if (!needCreateFilterListBasedOnFields()) {
|
||||
// Fetch all the columns. No need of a filter.
|
||||
return null;
|
||||
}
|
||||
FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE);
|
||||
FilterList infoColFamilyList = new FilterList();
|
||||
// By default fetch everything in INFO column family.
|
||||
FamilyFilter infoColumnFamily = new FamilyFilter(CompareOp.EQUAL,
|
||||
new BinaryComparator(SubApplicationColumnFamily.INFO.getBytes()));
|
||||
infoColFamilyList.addFilter(infoColumnFamily);
|
||||
if (fetchPartialColsFromInfoFamily()) {
|
||||
// We can fetch only some of the columns from info family.
|
||||
infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily());
|
||||
} else {
|
||||
// Exclude column prefixes in info column family which are not required
|
||||
// based on fields to retrieve.
|
||||
excludeFieldsFromInfoColFamily(infoColFamilyList);
|
||||
}
|
||||
listBasedOnFields.addFilter(infoColFamilyList);
|
||||
updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields);
|
||||
return listBasedOnFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void validateParams() {
|
||||
Preconditions.checkNotNull(getContext(), "context shouldn't be null");
|
||||
Preconditions.checkNotNull(getDataToRetrieve(),
|
||||
"data to retrieve shouldn't be null");
|
||||
Preconditions.checkNotNull(getContext().getClusterId(),
|
||||
"clusterId shouldn't be null");
|
||||
Preconditions.checkNotNull(getContext().getDoAsUser(),
|
||||
"DoAsUser shouldn't be null");
|
||||
Preconditions.checkNotNull(getContext().getEntityType(),
|
||||
"entityType shouldn't be null");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void augmentParams(Configuration hbaseConf, Connection conn)
|
||||
throws IOException {
|
||||
getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve();
|
||||
createFiltersIfNull();
|
||||
}
|
||||
|
||||
private void setMetricsTimeRange(Query query) {
|
||||
// Set time range for metric values.
|
||||
HBaseTimelineStorageUtils.setMetricsTimeRange(query,
|
||||
SubApplicationColumnFamily.METRICS.getBytes(),
|
||||
getDataToRetrieve().getMetricsTimeBegin(),
|
||||
getDataToRetrieve().getMetricsTimeEnd());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ResultScanner getResults(Configuration hbaseConf, Connection conn,
|
||||
FilterList filterList) throws IOException {
|
||||
|
||||
// Scan through part of the table to find the entities belong to one app
|
||||
// and one type
|
||||
Scan scan = new Scan();
|
||||
TimelineReaderContext context = getContext();
|
||||
if (context.getDoAsUser() == null) {
|
||||
throw new BadRequestException("Invalid user!");
|
||||
}
|
||||
|
||||
RowKeyPrefix<SubApplicationRowKey> subApplicationRowKeyPrefix = null;
|
||||
// default mode, will always scans from beginning of entity type.
|
||||
if (getFilters() == null || getFilters().getFromId() == null) {
|
||||
subApplicationRowKeyPrefix = new SubApplicationRowKeyPrefix(
|
||||
context.getDoAsUser(), context.getClusterId(),
|
||||
context.getEntityType(), null, null, null);
|
||||
scan.setRowPrefixFilter(subApplicationRowKeyPrefix.getRowKeyPrefix());
|
||||
} else { // pagination mode, will scan from given entityIdPrefix!enitityId
|
||||
|
||||
SubApplicationRowKey entityRowKey = null;
|
||||
try {
|
||||
entityRowKey = SubApplicationRowKey
|
||||
.parseRowKeyFromString(getFilters().getFromId());
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new BadRequestException("Invalid filter fromid is provided.");
|
||||
}
|
||||
if (!context.getClusterId().equals(entityRowKey.getClusterId())) {
|
||||
throw new BadRequestException(
|
||||
"fromid doesn't belong to clusterId=" + context.getClusterId());
|
||||
}
|
||||
|
||||
// set start row
|
||||
scan.setStartRow(entityRowKey.getRowKey());
|
||||
|
||||
// get the bytes for stop row
|
||||
subApplicationRowKeyPrefix = new SubApplicationRowKeyPrefix(
|
||||
context.getDoAsUser(), context.getClusterId(),
|
||||
context.getEntityType(), null, null, null);
|
||||
|
||||
// set stop row
|
||||
scan.setStopRow(
|
||||
HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix(
|
||||
subApplicationRowKeyPrefix.getRowKeyPrefix()));
|
||||
|
||||
// set page filter to limit. This filter has to set only in pagination
|
||||
// mode.
|
||||
filterList.addFilter(new PageFilter(getFilters().getLimit()));
|
||||
}
|
||||
setMetricsTimeRange(scan);
|
||||
scan.setMaxVersions(getDataToRetrieve().getMetricsLimit());
|
||||
if (filterList != null && !filterList.getFilters().isEmpty()) {
|
||||
scan.setFilter(filterList);
|
||||
}
|
||||
return getTable().getResultScanner(hbaseConf, conn, scan);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Result getResult(Configuration hbaseConf, Connection conn,
|
||||
FilterList filterList) throws IOException {
|
||||
throw new UnsupportedOperationException(
|
||||
"we don't support a single entity query");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TimelineEntity parseEntity(Result result) throws IOException {
|
||||
if (result == null || result.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
TimelineEntity entity = new TimelineEntity();
|
||||
SubApplicationRowKey parseRowKey =
|
||||
SubApplicationRowKey.parseRowKey(result.getRow());
|
||||
entity.setType(parseRowKey.getEntityType());
|
||||
entity.setId(parseRowKey.getEntityId());
|
||||
entity.setIdPrefix(parseRowKey.getEntityIdPrefix().longValue());
|
||||
|
||||
TimelineEntityFilters filters = getFilters();
|
||||
// fetch created time
|
||||
Long createdTime =
|
||||
(Long) SubApplicationColumn.CREATED_TIME.readResult(result);
|
||||
entity.setCreatedTime(createdTime);
|
||||
|
||||
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
|
||||
// fetch is related to entities and match isRelatedTo filter. If isRelatedTo
|
||||
// filters do not match, entity would be dropped. We have to match filters
|
||||
// locally as relevant HBase filters to filter out rows on the basis of
|
||||
// isRelatedTo are not set in HBase scan.
|
||||
boolean checkIsRelatedTo =
|
||||
filters.getIsRelatedTo() != null
|
||||
&& filters.getIsRelatedTo().getFilterList().size() > 0;
|
||||
if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) {
|
||||
readRelationship(entity, result, SubApplicationColumnPrefix.IS_RELATED_TO,
|
||||
true);
|
||||
if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity,
|
||||
filters.getIsRelatedTo())) {
|
||||
return null;
|
||||
}
|
||||
if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
|
||||
entity.getIsRelatedToEntities().clear();
|
||||
}
|
||||
}
|
||||
|
||||
// fetch relates to entities and match relatesTo filter. If relatesTo
|
||||
// filters do not match, entity would be dropped. We have to match filters
|
||||
// locally as relevant HBase filters to filter out rows on the basis of
|
||||
// relatesTo are not set in HBase scan.
|
||||
boolean checkRelatesTo =
|
||||
!isSingleEntityRead() && filters.getRelatesTo() != null
|
||||
&& filters.getRelatesTo().getFilterList().size() > 0;
|
||||
if (hasField(fieldsToRetrieve, Field.RELATES_TO) || checkRelatesTo) {
|
||||
readRelationship(entity, result, SubApplicationColumnPrefix.RELATES_TO,
|
||||
false);
|
||||
if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity,
|
||||
filters.getRelatesTo())) {
|
||||
return null;
|
||||
}
|
||||
if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
|
||||
entity.getRelatesToEntities().clear();
|
||||
}
|
||||
}
|
||||
|
||||
// fetch info if fieldsToRetrieve contains INFO or ALL.
|
||||
if (hasField(fieldsToRetrieve, Field.INFO)) {
|
||||
readKeyValuePairs(entity, result, SubApplicationColumnPrefix.INFO, false);
|
||||
}
|
||||
|
||||
// fetch configs if fieldsToRetrieve contains CONFIGS or ALL.
|
||||
if (hasField(fieldsToRetrieve, Field.CONFIGS)) {
|
||||
readKeyValuePairs(entity, result, SubApplicationColumnPrefix.CONFIG,
|
||||
true);
|
||||
}
|
||||
|
||||
// fetch events and match event filters if they exist. If event filters do
|
||||
// not match, entity would be dropped. We have to match filters locally
|
||||
// as relevant HBase filters to filter out rows on the basis of events
|
||||
// are not set in HBase scan.
|
||||
boolean checkEvents =
|
||||
!isSingleEntityRead() && filters.getEventFilters() != null
|
||||
&& filters.getEventFilters().getFilterList().size() > 0;
|
||||
if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) {
|
||||
readEvents(entity, result, SubApplicationColumnPrefix.EVENT);
|
||||
if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity,
|
||||
filters.getEventFilters())) {
|
||||
return null;
|
||||
}
|
||||
if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
|
||||
entity.getEvents().clear();
|
||||
}
|
||||
}
|
||||
|
||||
// fetch metrics if fieldsToRetrieve contains METRICS or ALL.
|
||||
if (hasField(fieldsToRetrieve, Field.METRICS)) {
|
||||
readMetrics(entity, result, SubApplicationColumnPrefix.METRIC);
|
||||
}
|
||||
|
||||
entity.getInfo().put(TimelineReaderUtils.FROMID_KEY,
|
||||
parseRowKey.getRowKeyAsString());
|
||||
return entity;
|
||||
}
|
||||
|
||||
}
|
|
@ -82,6 +82,9 @@ public final class TimelineEntityReaderFactory {
|
|||
YARN_FLOW_RUN.matches(context.getEntityType())) {
|
||||
return new FlowRunEntityReader(context, filters, dataToRetrieve);
|
||||
} else {
|
||||
if (context.getDoAsUser() != null) {
|
||||
return new SubApplicationEntityReader(context, filters, dataToRetrieve);
|
||||
}
|
||||
// assume we're dealing with a generic entity read
|
||||
return new GenericEntityReader(context, filters, dataToRetrieve);
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ public class TimelineReaderContext extends TimelineContext {
|
|||
private String entityType;
|
||||
private String entityId;
|
||||
private Long entityIdPrefix;
|
||||
private String doAsUser;
|
||||
public TimelineReaderContext(String clusterId, String userId, String flowName,
|
||||
Long flowRunId, String appId, String entityType, String entityId) {
|
||||
super(clusterId, userId, flowName, flowRunId, appId);
|
||||
|
@ -46,10 +47,18 @@ public class TimelineReaderContext extends TimelineContext {
|
|||
this.entityIdPrefix = entityIdPrefix;
|
||||
}
|
||||
|
||||
public TimelineReaderContext(String clusterId, String userId, String flowName,
|
||||
Long flowRunId, String appId, String entityType, Long entityIdPrefix,
|
||||
String entityId, String doasUser) {
|
||||
this(clusterId, userId, flowName, flowRunId, appId, entityType, entityId);
|
||||
this.entityIdPrefix = entityIdPrefix;
|
||||
this.doAsUser = doasUser;
|
||||
}
|
||||
|
||||
public TimelineReaderContext(TimelineReaderContext other) {
|
||||
this(other.getClusterId(), other.getUserId(), other.getFlowName(),
|
||||
other.getFlowRunId(), other.getAppId(), other.getEntityType(),
|
||||
other.getEntityIdPrefix(), other.getEntityId());
|
||||
other.getEntityIdPrefix(), other.getEntityId(), other.getDoAsUser());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -113,4 +122,12 @@ public class TimelineReaderContext extends TimelineContext {
|
|||
public void setEntityIdPrefix(Long entityIdPrefix) {
|
||||
this.entityIdPrefix = entityIdPrefix;
|
||||
}
|
||||
|
||||
public String getDoAsUser() {
|
||||
return doAsUser;
|
||||
}
|
||||
|
||||
public void setDoAsUser(String doAsUser) {
|
||||
this.doAsUser = doAsUser;
|
||||
}
|
||||
}
|
|
@ -111,8 +111,13 @@ public class TimelineReaderManager extends AbstractService {
|
|||
context.setEntityType(entity.getType());
|
||||
context.setEntityIdPrefix(entity.getIdPrefix());
|
||||
context.setEntityId(entity.getId());
|
||||
entity.setUID(TimelineReaderUtils.UID_KEY,
|
||||
TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context));
|
||||
if (context.getDoAsUser() != null) {
|
||||
entity.setUID(TimelineReaderUtils.UID_KEY,
|
||||
TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.encodeUID(context));
|
||||
} else {
|
||||
entity.setUID(TimelineReaderUtils.UID_KEY,
|
||||
TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -3245,4 +3245,162 @@ public class TimelineReaderWebServices {
|
|||
" (Took " + (endTime - startTime) + " ms.)");
|
||||
return results;
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/users/{userid}/entities/{entitytype}")
|
||||
@Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
|
||||
public Set<TimelineEntity> getSubAppEntities(
|
||||
@Context HttpServletRequest req,
|
||||
@Context HttpServletResponse res,
|
||||
@PathParam("userid") String userId,
|
||||
@PathParam("entitytype") String entityType,
|
||||
@QueryParam("limit") String limit,
|
||||
@QueryParam("createdtimestart") String createdTimeStart,
|
||||
@QueryParam("createdtimeend") String createdTimeEnd,
|
||||
@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("confstoretrieve") String confsToRetrieve,
|
||||
@QueryParam("metricstoretrieve") String metricsToRetrieve,
|
||||
@QueryParam("fields") String fields,
|
||||
@QueryParam("metricslimit") String metricsLimit,
|
||||
@QueryParam("metricstimestart") String metricsTimeStart,
|
||||
@QueryParam("metricstimeend") String metricsTimeEnd,
|
||||
@QueryParam("fromid") String fromId) {
|
||||
return getSubAppEntities(req, res, null, userId, entityType, limit,
|
||||
createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, infofilters,
|
||||
conffilters, metricfilters, eventfilters, confsToRetrieve,
|
||||
metricsToRetrieve, fields, metricsLimit, metricsTimeStart,
|
||||
metricsTimeEnd, fromId);
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/clusters/{clusterid}/users/{userid}/entities/{entitytype}")
|
||||
@Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
|
||||
public Set<TimelineEntity> getSubAppEntities(
|
||||
@Context HttpServletRequest req,
|
||||
@Context HttpServletResponse res,
|
||||
@PathParam("clusterid") String clusterId,
|
||||
@PathParam("userid") String userId,
|
||||
@PathParam("entitytype") String entityType,
|
||||
@QueryParam("limit") String limit,
|
||||
@QueryParam("createdtimestart") String createdTimeStart,
|
||||
@QueryParam("createdtimeend") String createdTimeEnd,
|
||||
@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("confstoretrieve") String confsToRetrieve,
|
||||
@QueryParam("metricstoretrieve") String metricsToRetrieve,
|
||||
@QueryParam("fields") String fields,
|
||||
@QueryParam("metricslimit") String metricsLimit,
|
||||
@QueryParam("metricstimestart") String metricsTimeStart,
|
||||
@QueryParam("metricstimeend") String metricsTimeEnd,
|
||||
@QueryParam("fromid") String fromId) {
|
||||
String url = req.getRequestURI() +
|
||||
(req.getQueryString() == null ? "" :
|
||||
QUERY_STRING_SEP + req.getQueryString());
|
||||
UserGroupInformation callerUGI =
|
||||
TimelineReaderWebServicesUtils.getUser(req);
|
||||
LOG.info("Received URL " + url + " from user " +
|
||||
TimelineReaderWebServicesUtils.getUserName(callerUGI));
|
||||
long startTime = Time.monotonicNow();
|
||||
init(res);
|
||||
TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
|
||||
Set<TimelineEntity> entities = null;
|
||||
try {
|
||||
TimelineReaderContext context =
|
||||
TimelineReaderWebServicesUtils.createTimelineReaderContext(clusterId,
|
||||
null, null, null, null, entityType, null, null, userId);
|
||||
entities = timelineReaderManager.getEntities(context,
|
||||
TimelineReaderWebServicesUtils.createTimelineEntityFilters(
|
||||
limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
|
||||
infofilters, conffilters, metricfilters, eventfilters,
|
||||
fromId),
|
||||
TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
|
||||
confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
|
||||
metricsTimeStart, metricsTimeEnd));
|
||||
} 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;
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/users/{userid}/entities/{entitytype}/{entityid}")
|
||||
@Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
|
||||
public Set<TimelineEntity> getSubAppEntities(@Context HttpServletRequest req,
|
||||
@Context HttpServletResponse res, @PathParam("userid") String userId,
|
||||
@PathParam("entitytype") String entityType,
|
||||
@PathParam("entityid") String entityId,
|
||||
@QueryParam("confstoretrieve") String confsToRetrieve,
|
||||
@QueryParam("metricstoretrieve") String metricsToRetrieve,
|
||||
@QueryParam("fields") String fields,
|
||||
@QueryParam("metricslimit") String metricsLimit,
|
||||
@QueryParam("metricstimestart") String metricsTimeStart,
|
||||
@QueryParam("metricstimeend") String metricsTimeEnd,
|
||||
@QueryParam("entityidprefix") String entityIdPrefix) {
|
||||
return getSubAppEntities(req, res, null, userId, entityType, entityId,
|
||||
confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
|
||||
metricsTimeStart, metricsTimeEnd, entityIdPrefix);
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/clusters/{clusterid}/users/{userid}/entities/{entitytype}/{entityid}")
|
||||
@Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
|
||||
public Set<TimelineEntity> getSubAppEntities(@Context HttpServletRequest req,
|
||||
@Context HttpServletResponse res,
|
||||
@PathParam("clusterid") String clusterId,
|
||||
@PathParam("userid") String userId,
|
||||
@PathParam("entitytype") String entityType,
|
||||
@PathParam("entityid") String entityId,
|
||||
@QueryParam("confstoretrieve") String confsToRetrieve,
|
||||
@QueryParam("metricstoretrieve") String metricsToRetrieve,
|
||||
@QueryParam("fields") String fields,
|
||||
@QueryParam("metricslimit") String metricsLimit,
|
||||
@QueryParam("metricstimestart") String metricsTimeStart,
|
||||
@QueryParam("metricstimeend") String metricsTimeEnd,
|
||||
@QueryParam("entityidprefix") String entityIdPrefix) {
|
||||
String url = req.getRequestURI() + (req.getQueryString() == null ? ""
|
||||
: QUERY_STRING_SEP + req.getQueryString());
|
||||
UserGroupInformation callerUGI =
|
||||
TimelineReaderWebServicesUtils.getUser(req);
|
||||
LOG.info("Received URL " + url + " from user "
|
||||
+ TimelineReaderWebServicesUtils.getUserName(callerUGI));
|
||||
long startTime = Time.monotonicNow();
|
||||
init(res);
|
||||
TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
|
||||
Set<TimelineEntity> entities = null;
|
||||
try {
|
||||
TimelineReaderContext context = TimelineReaderWebServicesUtils
|
||||
.createTimelineReaderContext(clusterId, null, null, null, null,
|
||||
entityType, entityIdPrefix, entityId, userId);
|
||||
entities = timelineReaderManager.getEntities(context,
|
||||
new TimelineEntityFilters.Builder().build(),
|
||||
TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
|
||||
confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
|
||||
metricsTimeStart, metricsTimeEnd));
|
||||
} catch (Exception e) {
|
||||
handleException(e, url, startTime, "");
|
||||
}
|
||||
long endTime = Time.monotonicNow();
|
||||
if (entities == null) {
|
||||
entities = Collections.emptySet();
|
||||
}
|
||||
LOG.info(
|
||||
"Processed URL " + url + " (Took " + (endTime - startTime) + " ms.)");
|
||||
return entities;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,6 +57,16 @@ public final class TimelineReaderWebServicesUtils {
|
|||
parseStr(entityType), parseLongStr(entityIdPrefix), parseStr(entityId));
|
||||
}
|
||||
|
||||
static TimelineReaderContext createTimelineReaderContext(String clusterId,
|
||||
String userId, String flowName, String flowRunId, String appId,
|
||||
String entityType, String entityIdPrefix, String entityId,
|
||||
String doAsUser) {
|
||||
return new TimelineReaderContext(parseStr(clusterId), parseStr(userId),
|
||||
parseStr(flowName), parseLongStr(flowRunId), parseStr(appId),
|
||||
parseStr(entityType), parseLongStr(entityIdPrefix), parseStr(entityId),
|
||||
parseStr(doAsUser));
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse the passed filters represented as strings and convert them into a
|
||||
* {@link TimelineEntityFilters} object.
|
||||
|
|
|
@ -137,6 +137,41 @@ enum TimelineUIDConverter {
|
|||
}
|
||||
},
|
||||
|
||||
// Sub Application Entity UID should contain cluster, user, entity type and
|
||||
// entity id
|
||||
SUB_APPLICATION_ENTITY_UID {
|
||||
@Override
|
||||
String encodeUID(TimelineReaderContext context) {
|
||||
if (context == null) {
|
||||
return null;
|
||||
}
|
||||
if (context.getClusterId() == null || context.getDoAsUser() == null
|
||||
|| context.getEntityType() == null || context.getEntityId() == null) {
|
||||
return null;
|
||||
}
|
||||
String[] entityTupleArr = {context.getClusterId(), context.getDoAsUser(),
|
||||
context.getEntityType(), context.getEntityIdPrefix().toString(),
|
||||
context.getEntityId()};
|
||||
return joinAndEscapeUIDParts(entityTupleArr);
|
||||
}
|
||||
|
||||
@Override
|
||||
TimelineReaderContext decodeUID(String uId) throws Exception {
|
||||
if (uId == null) {
|
||||
return null;
|
||||
}
|
||||
List<String> entityTupleList = splitUID(uId);
|
||||
if (entityTupleList.size() == 5) {
|
||||
// Flow information exists.
|
||||
return new TimelineReaderContext(entityTupleList.get(0), null, null,
|
||||
null, null, entityTupleList.get(2),
|
||||
Long.parseLong(entityTupleList.get(3)), entityTupleList.get(4),
|
||||
entityTupleList.get(1));
|
||||
}
|
||||
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).
|
||||
|
|
|
@ -68,6 +68,15 @@ public class TestTimelineUIDConverter {
|
|||
"54321!container_1111111111_1111_01_000001", uid);
|
||||
assertEquals(
|
||||
context, TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uid));
|
||||
|
||||
context = new TimelineReaderContext("yarn_cluster", null, null, null, null,
|
||||
"YARN_CONTAINER", 54321L, "container_1111111111_1111_01_000001",
|
||||
"user1");
|
||||
uid = TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.encodeUID(context);
|
||||
assertEquals("yarn_cluster!user1!YARN_CONTAINER!"
|
||||
+ "54321!container_1111111111_1111_01_000001", uid);
|
||||
assertEquals(context,
|
||||
TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.decodeUID(uid));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
Loading…
Reference in New Issue