YARN-5170. Eliminate singleton converters and static method access. (Joep Rottinghuis via Varun Saxena)

This commit is contained in:
Varun Saxena 2016-06-10 21:18:05 +05:30 committed by Sangjin Lee
parent 3832795e3c
commit 892b193bd7
41 changed files with 1885 additions and 1634 deletions

View File

@ -70,12 +70,14 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
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.EntityRowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
import org.junit.After;
import org.junit.AfterClass;
@ -649,8 +651,9 @@ public class TestHBaseTimelineStorage {
infoMap.putAll(infoMap1);
// retrieve the row
byte[] rowKey =
ApplicationRowKey.getRowKey(cluster, user, flow, runid, appId);
ApplicationRowKey applicationRowKey =
new ApplicationRowKey(cluster, user, flow, runid, appId);
byte[] rowKey = applicationRowKey.getRowKey();
Get get = new Get(rowKey);
get.setMaxVersions(Integer.MAX_VALUE);
Connection conn = ConnectionFactory.createConnection(c1);
@ -674,7 +677,7 @@ public class TestHBaseTimelineStorage {
Map<String, Object> infoColumns =
ApplicationColumnPrefix.INFO.readResults(result,
StringKeyConverter.getInstance());
new StringKeyConverter());
assertEquals(infoMap, infoColumns);
// Remember isRelatedTo is of type Map<String, Set<String>>
@ -710,15 +713,16 @@ public class TestHBaseTimelineStorage {
}
}
KeyConverter<String> stringKeyConverter = new StringKeyConverter();
// Configuration
Map<String, Object> configColumns =
ApplicationColumnPrefix.CONFIG.readResults(result,
StringKeyConverter.getInstance());
ApplicationColumnPrefix.CONFIG
.readResults(result, stringKeyConverter);
assertEquals(conf, configColumns);
NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(
result, StringKeyConverter.getInstance());
ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result,
stringKeyConverter);
NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
matchMetrics(metricValues, metricMap);
@ -908,7 +912,8 @@ public class TestHBaseTimelineStorage {
// scan the table and see that entity exists
Scan s = new Scan();
byte[] startRow =
EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
.getRowKeyPrefix();
s.setStartRow(startRow);
s.setMaxVersions(Integer.MAX_VALUE);
Connection conn = ConnectionFactory.createConnection(c1);
@ -916,6 +921,7 @@ public class TestHBaseTimelineStorage {
int rowCount = 0;
int colCount = 0;
KeyConverter<String> stringKeyConverter = new StringKeyConverter();
for (Result result : scanner) {
if (result != null && !result.isEmpty()) {
rowCount++;
@ -936,7 +942,7 @@ public class TestHBaseTimelineStorage {
Map<String, Object> infoColumns =
EntityColumnPrefix.INFO.readResults(result,
StringKeyConverter.getInstance());
new StringKeyConverter());
assertEquals(infoMap, infoColumns);
// Remember isRelatedTo is of type Map<String, Set<String>>
@ -975,13 +981,12 @@ public class TestHBaseTimelineStorage {
// Configuration
Map<String, Object> configColumns =
EntityColumnPrefix.CONFIG.readResults(result,
StringKeyConverter.getInstance());
EntityColumnPrefix.CONFIG.readResults(result, stringKeyConverter);
assertEquals(conf, configColumns);
NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
EntityColumnPrefix.METRIC.readResultsWithTimestamps(
result, StringKeyConverter.getInstance());
EntityColumnPrefix.METRIC.readResultsWithTimestamps(result,
stringKeyConverter);
NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
matchMetrics(metricValues, metricMap);
@ -1116,8 +1121,9 @@ public class TestHBaseTimelineStorage {
hbi.stop();
// retrieve the row
byte[] rowKey =
ApplicationRowKey.getRowKey(cluster, user, flow, runid, appName);
ApplicationRowKey applicationRowKey =
new ApplicationRowKey(cluster, user, flow, runid, appName);
byte[] rowKey = applicationRowKey.getRowKey();
Get get = new Get(rowKey);
get.setMaxVersions(Integer.MAX_VALUE);
Connection conn = ConnectionFactory.createConnection(c1);
@ -1132,7 +1138,7 @@ public class TestHBaseTimelineStorage {
Map<EventColumnName, Object> eventsResult =
ApplicationColumnPrefix.EVENT.readResults(result,
EventColumnNameConverter.getInstance());
new EventColumnNameConverter());
// there should be only one event
assertEquals(1, eventsResult.size());
for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
@ -1212,7 +1218,8 @@ public class TestHBaseTimelineStorage {
String appName = ApplicationId.newInstance(System.currentTimeMillis() +
9000000L, 1).toString();
byte[] startRow =
EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
.getRowKeyPrefix();
hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
hbi.stop();
// scan the table and see that entity exists
@ -1234,7 +1241,7 @@ public class TestHBaseTimelineStorage {
Map<EventColumnName, Object> eventsResult =
EntityColumnPrefix.EVENT.readResults(result,
EventColumnNameConverter.getInstance());
new EventColumnNameConverter());
// there should be only one event
assertEquals(1, eventsResult.size());
for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {

View File

@ -158,7 +158,7 @@ public class TestHBaseStorageFlowActivity {
Table table1 = conn.getTable(TableName
.valueOf(FlowActivityTable.DEFAULT_TABLE_NAME));
byte[] startRow =
FlowActivityRowKey.getRowKey(cluster, minStartTs, user, flow);
new FlowActivityRowKey(cluster, minStartTs, user, flow).getRowKey();
Get g = new Get(startRow);
Result r1 = table1.get(g);
assertNotNull(r1);
@ -278,11 +278,12 @@ public class TestHBaseStorageFlowActivity {
Scan s = new Scan();
s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
byte[] startRow =
FlowActivityRowKey.getRowKey(cluster, appCreatedTime, user, flow);
new FlowActivityRowKey(cluster, appCreatedTime, user, flow).getRowKey();
s.setStartRow(startRow);
String clusterStop = cluster + "1";
byte[] stopRow =
FlowActivityRowKey.getRowKey(clusterStop, appCreatedTime, user, flow);
new FlowActivityRowKey(clusterStop, appCreatedTime, user, flow)
.getRowKey();
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
Table table1 = conn.getTable(TableName
@ -420,11 +421,11 @@ public class TestHBaseStorageFlowActivity {
Scan s = new Scan();
s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
byte[] startRow =
FlowActivityRowKey.getRowKey(cluster, appCreatedTime, user, flow);
new FlowActivityRowKey(cluster, appCreatedTime, user, flow).getRowKey();
s.setStartRow(startRow);
String clusterStop = cluster + "1";
byte[] stopRow =
FlowActivityRowKey.getRowKey(clusterStop, appCreatedTime, user, flow);
new FlowActivityRowKey(clusterStop, appCreatedTime, user, flow).getRowKey();
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
Table table1 = conn.getTable(TableName

View File

@ -59,8 +59,8 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilte
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl;
import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
@ -224,7 +224,7 @@ public class TestHBaseStorageFlowRun {
.valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
// scan the table and see that we get back the right min and max
// timestamps
byte[] startRow = FlowRunRowKey.getRowKey(cluster, user, flow, runid);
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
Get g = new Get(startRow);
g.addColumn(FlowRunColumnFamily.INFO.getBytes(),
FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes());
@ -354,10 +354,11 @@ public class TestHBaseStorageFlowRun {
long runid, Configuration c1) throws IOException {
Scan s = new Scan();
s.addFamily(FlowRunColumnFamily.INFO.getBytes());
byte[] startRow = FlowRunRowKey.getRowKey(cluster, user, flow, runid);
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
s.setStartRow(startRow);
String clusterStop = cluster + "1";
byte[] stopRow = FlowRunRowKey.getRowKey(clusterStop, user, flow, runid);
byte[] stopRow =
new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
Table table1 = conn.getTable(TableName
@ -629,7 +630,7 @@ public class TestHBaseStorageFlowRun {
.valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
// scan the table and see that we get back the right min and max
// timestamps
byte[] startRow = FlowRunRowKey.getRowKey(cluster, user, flow, runid);
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
Get g = new Get(startRow);
g.addColumn(FlowRunColumnFamily.INFO.getBytes(),
FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes());

View File

@ -19,24 +19,24 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertNotEquals;
import java.io.IOException;
import java.util.Map;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.ArrayList;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
@ -46,21 +46,21 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
/**
* Tests the FlowRun and FlowActivity Tables
@ -194,10 +194,11 @@ public class TestHBaseStorageFlowRunCompaction {
long runid, Configuration c1, int valueCount) throws IOException {
Scan s = new Scan();
s.addFamily(FlowRunColumnFamily.INFO.getBytes());
byte[] startRow = FlowRunRowKey.getRowKey(cluster, user, flow, runid);
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
s.setStartRow(startRow);
String clusterStop = cluster + "1";
byte[] stopRow = FlowRunRowKey.getRowKey(clusterStop, user, flow, runid);
byte[] stopRow =
new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
Table table1 = conn.getTable(TableName
@ -302,8 +303,9 @@ public class TestHBaseStorageFlowRunCompaction {
cell4Ts, Bytes.toBytes(cellValue4), tagByteArray);
currentColumnCells.add(c4);
List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
LongConverter.getInstance(), currentTimestamp);
List<Cell> cells =
fs.processSummationMajorCompaction(currentColumnCells,
new LongConverter(), currentTimestamp);
assertNotNull(cells);
// we should be getting back 4 cells
@ -387,8 +389,9 @@ public class TestHBaseStorageFlowRunCompaction {
cellTsNotFinal++;
}
List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
LongConverter.getInstance(), currentTimestamp);
List<Cell> cells =
fs.processSummationMajorCompaction(currentColumnCells,
new LongConverter(), currentTimestamp);
assertNotNull(cells);
// we should be getting back count + 1 cells
@ -489,8 +492,9 @@ public class TestHBaseStorageFlowRunCompaction {
cellTsNotFinal++;
}
List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
LongConverter.getInstance(), currentTimestamp);
List<Cell> cells =
fs.processSummationMajorCompaction(currentColumnCells,
new LongConverter(), currentTimestamp);
assertNotNull(cells);
// we should be getting back
@ -554,7 +558,7 @@ public class TestHBaseStorageFlowRunCompaction {
130L, Bytes.toBytes(cellValue2), tagByteArray);
currentColumnCells.add(c2);
List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
LongConverter.getInstance(), currentTimestamp);
new LongConverter(), currentTimestamp);
assertNotNull(cells);
// we should be getting back two cells
@ -602,7 +606,7 @@ public class TestHBaseStorageFlowRunCompaction {
currentColumnCells.add(c1);
List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
LongConverter.getInstance(), currentTimestamp);
new LongConverter(), currentTimestamp);
assertNotNull(cells);
// we should not get the same cell back
// but we get back the flow cell
@ -639,7 +643,7 @@ public class TestHBaseStorageFlowRunCompaction {
currentTimestamp, Bytes.toBytes(1110L), tagByteArray);
currentColumnCells.add(c1);
List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
LongConverter.getInstance(), currentTimestamp);
new LongConverter(), currentTimestamp);
assertNotNull(cells);
// we expect the same cell back
assertEquals(1, cells.size());
@ -653,15 +657,19 @@ public class TestHBaseStorageFlowRunCompaction {
FlowScanner fs = getFlowScannerForTestingCompaction();
long currentTimestamp = System.currentTimeMillis();
LongConverter longConverter = new LongConverter();
SortedSet<Cell> currentColumnCells = null;
List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
LongConverter.getInstance(), currentTimestamp);
List<Cell> cells =
fs.processSummationMajorCompaction(currentColumnCells, longConverter,
currentTimestamp);
assertNotNull(cells);
assertEquals(0, cells.size());
currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
cells = fs.processSummationMajorCompaction(currentColumnCells,
LongConverter.getInstance(), currentTimestamp);
cells =
fs.processSummationMajorCompaction(currentColumnCells, longConverter,
currentTimestamp);
assertNotNull(cells);
assertEquals(0, cells.size());
}

View File

@ -31,14 +31,9 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
import org.apache.hadoop.hbase.filter.QualifierFilter;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
@ -209,39 +204,6 @@ public final class TimelineFilterUtils {
return singleColValFilter;
}
private static <T> byte[] createColQualifierPrefix(ColumnPrefix<T> colPrefix,
String column) {
if (colPrefix == ApplicationColumnPrefix.EVENT ||
colPrefix == EntityColumnPrefix.EVENT) {
return EventColumnNameConverter.getInstance().encode(
new EventColumnName(column, null, null));
} else {
return StringKeyConverter.getInstance().encode(column);
}
}
/**
* Create a filter list of qualifier filters based on passed set of columns.
*
* @param <T> Describes the type of column prefix.
* @param colPrefix Column Prefix.
* @param columns set of column qualifiers.
* @return filter list.
*/
public static <T> FilterList createFiltersFromColumnQualifiers(
ColumnPrefix<T> colPrefix, Set<String> columns) {
FilterList list = new FilterList(Operator.MUST_PASS_ONE);
for (String column : columns) {
// For columns which have compound column qualifiers (eg. events), we need
// to include the required separator.
byte[] compoundColQual = createColQualifierPrefix(colPrefix, column);
list.addFilter(new QualifierFilter(CompareOp.EQUAL,
new BinaryPrefixComparator(
colPrefix.getColumnPrefixBytes(compoundColQual))));
}
return list;
}
/**
* Fetch columns from filter list containing exists and multivalue equality
* filters. This is done to fetch only required columns from back-end and

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
@ -45,11 +46,10 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlow
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
@ -86,6 +86,17 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
private TypedBufferedMutator<FlowActivityTable> flowActivityTable;
private TypedBufferedMutator<FlowRunTable> flowRunTable;
/**
* Used to convert strings key components to and from storage format.
*/
private final KeyConverter<String> stringKeyConverter =
new StringKeyConverter();
/**
* Used to convert Long key components to and from storage format.
*/
private final KeyConverter<Long> longKeyConverter = new LongKeyConverter();
public HBaseTimelineWriterImpl() {
super(HBaseTimelineWriterImpl.class.getName());
}
@ -138,12 +149,19 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
// if the entity is the application, the destination is the application
// table
boolean isApplication = TimelineStorageUtils.isApplicationEntity(te);
byte[] rowKey = isApplication ?
ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId,
appId) :
EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId,
te.getType(), te.getId());
boolean isApplication = isApplicationEntity(te);
byte[] rowKey;
if (isApplication) {
ApplicationRowKey applicationRowKey =
new ApplicationRowKey(clusterId, userId, flowName, flowRunId,
appId);
rowKey = applicationRowKey.getRowKey();
} else {
EntityRowKey entityRowKey =
new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
te.getType(), te.getId());
rowKey = entityRowKey.getRowKey();
}
storeInfo(rowKey, te, flowVersion, isApplication);
storeEvents(rowKey, te.getEvents(), isApplication);
@ -152,102 +170,101 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
storeRelations(rowKey, te, isApplication);
if (isApplication) {
TimelineEvent event = TimelineStorageUtils.getApplicationEvent(te,
ApplicationMetricsConstants.CREATED_EVENT_TYPE);
TimelineEvent event =
getApplicationEvent(te,
ApplicationMetricsConstants.CREATED_EVENT_TYPE);
FlowRunRowKey flowRunRowKey =
new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
if (event != null) {
onApplicationCreated(clusterId, userId, flowName, flowVersion,
flowRunId, appId, te, event.getTimestamp());
AppToFlowRowKey appToFlowRowKey =
new AppToFlowRowKey(clusterId, appId);
onApplicationCreated(flowRunRowKey, appToFlowRowKey, appId, userId,
flowVersion, te, event.getTimestamp());
}
// if it's an application entity, store metrics
storeFlowMetricsAppRunning(clusterId, userId, flowName, flowRunId,
appId, te);
storeFlowMetricsAppRunning(flowRunRowKey, appId, te);
// if application has finished, store it's finish time and write final
// values of all metrics
event = TimelineStorageUtils.getApplicationEvent(te,
event = getApplicationEvent(te,
ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
if (event != null) {
onApplicationFinished(clusterId, userId, flowName, flowVersion,
flowRunId, appId, te, event.getTimestamp());
onApplicationFinished(flowRunRowKey, flowVersion, appId, te,
event.getTimestamp());
}
}
}
return putStatus;
}
private void onApplicationCreated(String clusterId, String userId,
String flowName, String flowVersion, long flowRunId, String appId,
TimelineEntity te, long appCreatedTimeStamp) throws IOException {
// store in App to flow table
storeInAppToFlowTable(clusterId, userId, flowName, flowRunId, appId, te);
// store in flow run table
storeAppCreatedInFlowRunTable(clusterId, userId, flowName, flowVersion,
flowRunId, appId, te);
// store in flow activity table
storeInFlowActivityTable(clusterId, userId, flowName, flowVersion,
flowRunId, appId, appCreatedTimeStamp);
}
private void onApplicationCreated(FlowRunRowKey flowRunRowKey,
AppToFlowRowKey appToFlowRowKey, String appId, String userId,
String flowVersion, TimelineEntity te, long appCreatedTimeStamp)
throws IOException {
/*
* updates the {@link FlowActivityTable} with the Application TimelineEntity
* information
*/
private void storeInFlowActivityTable(String clusterId, String userId,
String flowName, String flowVersion, long flowRunId, String appId,
long activityTimeStamp) throws IOException {
byte[] rowKey = FlowActivityRowKey.getRowKey(clusterId, activityTimeStamp,
userId, flowName);
byte[] qualifier = LongKeyConverter.getInstance().encode(flowRunId);
FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier,
null, flowVersion,
String flowName = flowRunRowKey.getFlowName();
Long flowRunId = flowRunRowKey.getFlowRunId();
// store in App to flow table
byte[] rowKey = appToFlowRowKey.getRowKey();
AppToFlowColumn.FLOW_ID.store(rowKey, appToFlowTable, null, flowName);
AppToFlowColumn.FLOW_RUN_ID.store(rowKey, appToFlowTable, null, flowRunId);
AppToFlowColumn.USER_ID.store(rowKey, appToFlowTable, null, userId);
// store in flow run table
storeAppCreatedInFlowRunTable(flowRunRowKey, appId, te);
// store in flow activity table
byte[] flowActivityRowKeyBytes =
new FlowActivityRowKey(flowRunRowKey.getClusterId(),
appCreatedTimeStamp, flowRunRowKey.getUserId(), flowName)
.getRowKey();
byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId());
FlowActivityColumnPrefix.RUN_ID.store(flowActivityRowKeyBytes,
flowActivityTable, qualifier, null, flowVersion,
AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
}
/*
* updates the {@link FlowRunTable} with Application Created information
*/
private void storeAppCreatedInFlowRunTable(String clusterId, String userId,
String flowName, String flowVersion, long flowRunId, String appId,
TimelineEntity te) throws IOException {
byte[] rowKey = FlowRunRowKey.getRowKey(clusterId, userId, flowName,
flowRunId);
private void storeAppCreatedInFlowRunTable(FlowRunRowKey flowRunRowKey,
String appId, TimelineEntity te) throws IOException {
byte[] rowKey = flowRunRowKey.getRowKey();
FlowRunColumn.MIN_START_TIME.store(rowKey, flowRunTable, null,
te.getCreatedTime(),
AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
}
private void storeInAppToFlowTable(String clusterId, String userId,
String flowName, long flowRunId, String appId, TimelineEntity te)
throws IOException {
byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
AppToFlowColumn.FLOW_ID.store(rowKey, appToFlowTable, null, flowName);
AppToFlowColumn.FLOW_RUN_ID.store(rowKey, appToFlowTable, null, flowRunId);
AppToFlowColumn.USER_ID.store(rowKey, appToFlowTable, null, userId);
}
/*
* updates the {@link FlowRunTable} and {@link FlowActivityTable} when an
* application has finished
*/
private void onApplicationFinished(String clusterId, String userId,
String flowName, String flowVersion, long flowRunId, String appId,
TimelineEntity te, long appFinishedTimeStamp) throws IOException {
private void onApplicationFinished(FlowRunRowKey flowRunRowKey,
String flowVersion, String appId, TimelineEntity te,
long appFinishedTimeStamp) throws IOException {
// store in flow run table
storeAppFinishedInFlowRunTable(clusterId, userId, flowName, flowRunId,
appId, te, appFinishedTimeStamp);
storeAppFinishedInFlowRunTable(flowRunRowKey, appId, te,
appFinishedTimeStamp);
// indicate in the flow activity table that the app has finished
storeInFlowActivityTable(clusterId, userId, flowName, flowVersion,
flowRunId, appId, appFinishedTimeStamp);
byte[] rowKey =
new FlowActivityRowKey(flowRunRowKey.getClusterId(),
appFinishedTimeStamp, flowRunRowKey.getUserId(),
flowRunRowKey.getFlowName()).getRowKey();
byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId());
FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier,
null, flowVersion,
AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
}
/*
* Update the {@link FlowRunTable} with Application Finished information
*/
private void storeAppFinishedInFlowRunTable(String clusterId, String userId,
String flowName, long flowRunId, String appId, TimelineEntity te,
long appFinishedTimeStamp) throws IOException {
byte[] rowKey =
FlowRunRowKey.getRowKey(clusterId, userId, flowName, flowRunId);
private void storeAppFinishedInFlowRunTable(FlowRunRowKey flowRunRowKey,
String appId, TimelineEntity te, long appFinishedTimeStamp)
throws IOException {
byte[] rowKey = flowRunRowKey.getRowKey();
Attribute attributeAppId =
AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId);
FlowRunColumn.MAX_END_TIME.store(rowKey, flowRunTable, null,
@ -264,13 +281,11 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
/*
* Updates the {@link FlowRunTable} with Application Metrics
*/
private void storeFlowMetricsAppRunning(String clusterId, String userId,
String flowName, long flowRunId, String appId, TimelineEntity te)
throws IOException {
private void storeFlowMetricsAppRunning(FlowRunRowKey flowRunRowKey,
String appId, TimelineEntity te) throws IOException {
Set<TimelineMetric> metrics = te.getMetrics();
if (metrics != null) {
byte[] rowKey = FlowRunRowKey.getRowKey(clusterId, userId, flowName,
flowRunId);
byte[] rowKey = flowRunRowKey.getRowKey();
storeFlowMetrics(rowKey, metrics,
AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId),
AggregationOperation.SUM.getAttribute());
@ -280,8 +295,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
private void storeFlowMetrics(byte[] rowKey, Set<TimelineMetric> metrics,
Attribute... attributes) throws IOException {
for (TimelineMetric metric : metrics) {
byte[] metricColumnQualifier =
StringKeyConverter.getInstance().encode(metric.getId());
byte[] metricColumnQualifier = stringKeyConverter.encode(metric.getId());
Map<Long, Number> timeseries = metric.getValues();
for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
Long timestamp = timeseriesEntry.getKey();
@ -320,8 +334,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
String compoundValue =
Separator.VALUES.joinEncoded(connectedEntity.getValue());
columnPrefix.store(rowKey, table,
StringKeyConverter.getInstance().encode(connectedEntity.getKey()),
null, compoundValue);
stringKeyConverter.encode(connectedEntity.getKey()), null,
compoundValue);
}
}
@ -341,7 +355,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
if (info != null) {
for (Map.Entry<String, Object> entry : info.entrySet()) {
ApplicationColumnPrefix.INFO.store(rowKey, applicationTable,
StringKeyConverter.getInstance().encode(entry.getKey()), null,
stringKeyConverter.encode(entry.getKey()), null,
entry.getValue());
}
}
@ -355,7 +369,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
if (info != null) {
for (Map.Entry<String, Object> entry : info.entrySet()) {
EntityColumnPrefix.INFO.store(rowKey, entityTable,
StringKeyConverter.getInstance().encode(entry.getKey()), null,
stringKeyConverter.encode(entry.getKey()), null,
entry.getValue());
}
}
@ -371,8 +385,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
return;
}
for (Map.Entry<String, String> entry : config.entrySet()) {
byte[] configKey =
StringKeyConverter.getInstance().encode(entry.getKey());
byte[] configKey = stringKeyConverter.encode(entry.getKey());
if (isApplication) {
ApplicationColumnPrefix.CONFIG.store(rowKey, applicationTable,
configKey, null, entry.getValue());
@ -392,7 +405,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
if (metrics != null) {
for (TimelineMetric metric : metrics) {
byte[] metricColumnQualifier =
StringKeyConverter.getInstance().encode(metric.getId());
stringKeyConverter.encode(metric.getId());
Map<Long, Number> timeseries = metric.getValues();
for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
Long timestamp = timeseriesEntry.getKey();
@ -425,12 +438,11 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
"! Using the current timestamp");
eventTimestamp = System.currentTimeMillis();
}
EventColumnNameConverter converter =
EventColumnNameConverter.getInstance();
Map<String, Object> eventInfo = event.getInfo();
if ((eventInfo == null) || (eventInfo.size() == 0)) {
byte[] columnQualifierBytes = converter.encode(
new EventColumnName(eventId, eventTimestamp, null));
byte[] columnQualifierBytes =
new EventColumnName(eventId, eventTimestamp, null)
.getColumnQualifier();
if (isApplication) {
ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
columnQualifierBytes, null, Separator.EMPTY_BYTES);
@ -441,9 +453,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
} else {
for (Map.Entry<String, Object> info : eventInfo.entrySet()) {
// eventId=infoKey
byte[] columnQualifierBytes = converter.encode(
new EventColumnName(eventId, eventTimestamp,
info.getKey()));
byte[] columnQualifierBytes =
new EventColumnName(eventId, eventTimestamp, info.getKey())
.getColumnQualifier();
if (isApplication) {
ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
columnQualifierBytes, null, info.getValue());
@ -459,12 +471,56 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
}
}
/**
* Checks if the input TimelineEntity object is an ApplicationEntity.
*
* @param te TimelineEntity object.
* @return true if input is an ApplicationEntity, false otherwise
*/
static boolean isApplicationEntity(TimelineEntity te) {
return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
}
/**
* @param te TimelineEntity object.
* @param eventId event with this id needs to be fetched
* @return TimelineEvent if TimelineEntity contains the desired event.
*/
private static TimelineEvent getApplicationEvent(TimelineEntity te,
String eventId) {
if (isApplicationEntity(te)) {
for (TimelineEvent event : te.getEvents()) {
if (event.getId().equals(eventId)) {
return event;
}
}
}
return null;
}
/*
* (non-Javadoc)
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage
* .TimelineWriter#aggregate
* (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity,
* org.apache
* .hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack)
*/
@Override
public TimelineWriteResponse aggregate(TimelineEntity data,
TimelineAggregationTrack track) throws IOException {
return null;
}
/*
* (non-Javadoc)
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter#flush
* ()
*/
@Override
public void flush() throws IOException {
// flush all buffered mutators

View File

@ -45,7 +45,7 @@ public enum ApplicationColumn implements Column<ApplicationTable> {
* When the application was created.
*/
CREATED_TIME(ApplicationColumnFamily.INFO, "created_time",
LongConverter.getInstance()),
new LongConverter()),
/**
* The version of the flow that this app belongs to.

View File

@ -67,8 +67,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
/**
* Metrics are stored with the metric name as the column name.
*/
METRIC(ApplicationColumnFamily.METRICS, null,
LongConverter.getInstance());
METRIC(ApplicationColumnFamily.METRICS, null, new LongConverter());
private final ColumnHelper<ApplicationTable> column;
private final ColumnFamily<ApplicationTable> columnFamily;

View File

@ -18,6 +18,12 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.application;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
/**
* Represents a rowkey for the application table.
*/
@ -27,6 +33,8 @@ public class ApplicationRowKey {
private final String flowName;
private final Long flowRunId;
private final String appId;
private final KeyConverter<ApplicationRowKey> appRowKeyConverter =
new ApplicationRowKeyConverter();
public ApplicationRowKey(String clusterId, String userId, String flowName,
Long flowRunId, String appId) {
@ -57,61 +65,142 @@ public class ApplicationRowKey {
return appId;
}
/**
* Constructs a row key prefix for the application table as follows:
* {@code clusterId!userName!flowName!}.
*
* @param clusterId Cluster Id.
* @param userId User Id.
* @param flowName Flow Name.
* @return byte array with the row key prefix
*/
public static byte[] getRowKeyPrefix(String clusterId, String userId,
String flowName) {
return ApplicationRowKeyConverter.getInstance().encode(
new ApplicationRowKey(clusterId, userId, flowName, null, null));
}
/**
* Constructs a row key prefix for the application table as follows:
* {@code clusterId!userName!flowName!flowRunId!}.
*
* @param clusterId Cluster Id.
* @param userId User Id.
* @param flowName Flow Name.
* @param flowRunId Run Id for the flow.
* @return byte array with the row key prefix
*/
public static byte[] getRowKeyPrefix(String clusterId, String userId,
String flowName, Long flowRunId) {
return ApplicationRowKeyConverter.getInstance().encode(
new ApplicationRowKey(clusterId, userId, flowName, flowRunId, null));
}
/**
* Constructs a row key for the application table as follows:
* {@code clusterId!userName!flowName!flowRunId!AppId}.
*
* @param clusterId Cluster Id.
* @param userId User Id.
* @param flowName Flow Name.
* @param flowRunId Run Id for the flow.
* @param appId App Id.
* @return byte array with the row key
*/
public static byte[] getRowKey(String clusterId, String userId,
String flowName, Long flowRunId, String appId) {
return ApplicationRowKeyConverter.getInstance().encode(
new ApplicationRowKey(clusterId, userId, flowName, flowRunId, appId));
public byte[] getRowKey() {
return appRowKeyConverter.encode(this);
}
/**
* Given the raw row key as bytes, returns the row key as an object.
*
* @param rowKey Byte representation of row key.
* @param rowKey Byte representation of row key.
* @return An <cite>ApplicationRowKey</cite> object.
*/
public static ApplicationRowKey parseRowKey(byte[] rowKey) {
return ApplicationRowKeyConverter.getInstance().decode(rowKey);
return new ApplicationRowKeyConverter().decode(rowKey);
}
/**
* Encodes and decodes row key for application table. The row key is of the
* form: clusterId!userName!flowName!flowRunId!appId. flowRunId is a long,
* appId is encoded and decoded using {@link AppIdKeyConverter} and rest are
* strings.
* <p>
*/
final private static class ApplicationRowKeyConverter implements
KeyConverter<ApplicationRowKey> {
private final KeyConverter<String> appIDKeyConverter =
new AppIdKeyConverter();
/**
* Intended for use in ApplicationRowKey only.
*/
private ApplicationRowKeyConverter() {
}
/**
* Application row key is of the form
* clusterId!userName!flowName!flowRunId!appId with each segment separated
* by !. The sizes below indicate sizes of each one of these segements in
* sequence. clusterId, userName and flowName are strings. flowrunId is a
* long hence 8 bytes in size. app id is represented as 12 bytes with
* cluster timestamp part of appid takes 8 bytes(long) and seq id takes 4
* bytes(int). Strings are variable in size (i.e. end whenever separator is
* encountered). This is used while decoding and helps in determining where
* to split.
*/
private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
AppIdKeyConverter.getKeySize() };
/*
* (non-Javadoc)
*
* Encodes ApplicationRowKey object into a byte array with each
* component/field in ApplicationRowKey separated by Separator#QUALIFIERS.
* This leads to an application table row key of the form
* clusterId!userName!flowName!flowRunId!appId If flowRunId in passed
* ApplicationRowKey object is null (and the fields preceding it i.e.
* clusterId, userId and flowName are not null), this returns a row key
* prefix of the form clusterId!userName!flowName! and if appId in
* ApplicationRowKey is null (other 4 components all are not null), this
* returns a row key prefix of the form
* clusterId!userName!flowName!flowRunId! flowRunId is inverted while
* encoding as it helps maintain a descending order for row keys in the
* application table.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#encode(java.lang.Object)
*/
@Override
public byte[] encode(ApplicationRowKey rowKey) {
byte[] cluster =
Separator.encode(rowKey.getClusterId(), Separator.SPACE,
Separator.TAB, Separator.QUALIFIERS);
byte[] user =
Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS);
byte[] flow =
Separator.encode(rowKey.getFlowName(), Separator.SPACE,
Separator.TAB, Separator.QUALIFIERS);
byte[] first = Separator.QUALIFIERS.join(cluster, user, flow);
// Note that flowRunId is a long, so we can't encode them all at the same
// time.
if (rowKey.getFlowRunId() == null) {
return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
}
byte[] second =
Bytes.toBytes(LongConverter.invertLong(
rowKey.getFlowRunId()));
if (rowKey.getAppId() == null || rowKey.getAppId().isEmpty()) {
return Separator.QUALIFIERS.join(first, second, Separator.EMPTY_BYTES);
}
byte[] third = appIDKeyConverter.encode(rowKey.getAppId());
return Separator.QUALIFIERS.join(first, second, third);
}
/*
* (non-Javadoc)
*
* Decodes an application row key of the form
* clusterId!userName!flowName!flowRunId!appId represented in byte format
* and converts it into an ApplicationRowKey object.flowRunId is inverted
* while decoding as it was inverted while encoding.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#decode(byte[])
*/
@Override
public ApplicationRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 5) {
throw new IllegalArgumentException("the row key is not valid for "
+ "an application");
}
String clusterId =
Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String userId =
Separator.decode(Bytes.toString(rowKeyComponents[1]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String flowName =
Separator.decode(Bytes.toString(rowKeyComponents[2]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
Long flowRunId =
LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3]));
String appId = appIDKeyConverter.decode(rowKeyComponents[4]);
return new ApplicationRowKey(clusterId, userId, flowName, flowRunId,
appId);
}
}
}

View File

@ -1,130 +0,0 @@
/**
* 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.application;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
/**
* Encodes and decodes row key for application table.
* The row key is of the form : clusterId!userName!flowName!flowRunId!appId.
* flowRunId is a long, appId is encoded and decoded using
* {@link AppIdKeyConverter} and rest are strings.
*/
public final class ApplicationRowKeyConverter implements
KeyConverter<ApplicationRowKey> {
private static final ApplicationRowKeyConverter INSTANCE =
new ApplicationRowKeyConverter();
public static ApplicationRowKeyConverter getInstance() {
return INSTANCE;
}
private ApplicationRowKeyConverter() {
}
// Application row key is of the form
// clusterId!userName!flowName!flowRunId!appId with each segment separated
// by !. The sizes below indicate sizes of each one of these segements in
// sequence. clusterId, userName and flowName are strings. flowrunId is a long
// hence 8 bytes in size. app id is represented as 12 bytes with cluster
// timestamp part of appid being 8 bytes(long) and seq id being 4 bytes(int).
// Strings are variable in size (i.e. end whenever separator is encountered).
// This is used while decoding and helps in determining where to split.
private static final int[] SEGMENT_SIZES = {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Bytes.SIZEOF_LONG, AppIdKeyConverter.getKeySize() };
/*
* (non-Javadoc)
*
* Encodes ApplicationRowKey object into a byte array with each
* component/field in ApplicationRowKey separated by Separator#QUALIFIERS.
* This leads to an application table row key of the form
* clusterId!userName!flowName!flowRunId!appId
* If flowRunId in passed ApplicationRowKey object is null (and the fields
* preceding it i.e. clusterId, userId and flowName are not null), this
* returns a row key prefix of the form clusterId!userName!flowName! and if
* appId in ApplicationRowKey is null (other 4 components are not null), this
* returns a row key prefix of the form clusterId!userName!flowName!flowRunId!
* flowRunId is inverted while encoding as it helps maintain a descending
* order for row keys in application table.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #encode(java.lang.Object)
*/
@Override
public byte[] encode(ApplicationRowKey rowKey) {
byte[] cluster = Separator.encode(rowKey.getClusterId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
byte[] user = Separator.encode(rowKey.getUserId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
byte[] flow = Separator.encode(rowKey.getFlowName(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
byte[] first = Separator.QUALIFIERS.join(cluster, user, flow);
// Note that flowRunId is a long, so we can't encode them all at the same
// time.
if (rowKey.getFlowRunId() == null) {
return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
}
byte[] second = Bytes.toBytes(
TimelineStorageUtils.invertLong(rowKey.getFlowRunId()));
if (rowKey.getAppId() == null || rowKey.getAppId().isEmpty()) {
return Separator.QUALIFIERS.join(first, second, Separator.EMPTY_BYTES);
}
byte[] third = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
return Separator.QUALIFIERS.join(first, second, third);
}
/*
* (non-Javadoc)
*
* Decodes an application row key of the form
* clusterId!userName!flowName!flowRunId!appId represented in byte format and
* converts it into an ApplicationRowKey object.flowRunId is inverted while
* decoding as it was inverted while encoding.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #decode(byte[])
*/
@Override
public ApplicationRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 5) {
throw new IllegalArgumentException("the row key is not valid for " +
"an application");
}
String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String userId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
Long flowRunId =
TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[4]);
return new ApplicationRowKey(clusterId, userId, flowName, flowRunId, appId);
}
}

View File

@ -0,0 +1,69 @@
/**
* 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.application;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
/**
* Represents a partial rowkey (without flowName or without flowName and
* flowRunId) for the application table.
*/
public class ApplicationRowKeyPrefix extends ApplicationRowKey implements
RowKeyPrefix<ApplicationRowKey> {
/**
* Creates a prefix which generates the following rowKeyPrefixes for the
* application table: {@code clusterId!userName!flowName!}.
*
* @param clusterId the cluster on which applications ran
* @param userId the user that ran applications
* @param flowName the name of the flow that was run by the user on the
* cluster
*/
public ApplicationRowKeyPrefix(String clusterId, String userId,
String flowName) {
super(clusterId, userId, flowName, null, null);
}
/**
* Creates a prefix which generates the following rowKeyPrefixes for the
* application table: {@code clusterId!userName!flowName!flowRunId!}.
*
* @param clusterId identifying the cluster
* @param userId identifying the user
* @param flowName identifying the flow
* @param flowRunId identifying the instance of this flow
*/
public ApplicationRowKeyPrefix(String clusterId, String userId,
String flowName, Long flowRunId) {
super(clusterId, userId, flowName, flowRunId, null);
}
/*
* (non-Javadoc)
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.application.
* RowKeyPrefix#getRowKeyPrefix()
*/
@Override
public byte[] getRowKeyPrefix() {
return super.getRowKey();
}
}

View File

@ -17,12 +17,19 @@
*/
package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
/**
* Represents a rowkey for the app_flow table.
*/
public class AppToFlowRowKey {
private final String clusterId;
private final String appId;
private final KeyConverter<AppToFlowRowKey> appToFlowRowKeyConverter =
new AppToFlowRowKeyConverter();
public AppToFlowRowKey(String clusterId, String appId) {
this.clusterId = clusterId;
@ -41,13 +48,10 @@ public class AppToFlowRowKey {
* Constructs a row key prefix for the app_flow table as follows:
* {@code clusterId!AppId}.
*
* @param clusterId Cluster Id.
* @param appId Application Id.
* @return byte array with the row key
*/
public static byte[] getRowKey(String clusterId, String appId) {
return AppToFlowRowKeyConverter.getInstance().encode(
new AppToFlowRowKey(clusterId, appId));
public byte[] getRowKey() {
return appToFlowRowKeyConverter.encode(this);
}
/**
@ -57,6 +61,83 @@ public class AppToFlowRowKey {
* @return an <cite>AppToFlowRowKey</cite> object.
*/
public static AppToFlowRowKey parseRowKey(byte[] rowKey) {
return AppToFlowRowKeyConverter.getInstance().decode(rowKey);
return new AppToFlowRowKeyConverter().decode(rowKey);
}
/**
* Encodes and decodes row key for app_flow table. The row key is of the form
* clusterId!appId. clusterId is a string and appId is encoded/decoded using
* {@link AppIdKeyConverter}.
* <p>
*/
final private static class AppToFlowRowKeyConverter implements
KeyConverter<AppToFlowRowKey> {
private final KeyConverter<String> appIDKeyConverter =
new AppIdKeyConverter();
/**
* Intended for use in AppToFlowRowKey only.
*/
private AppToFlowRowKeyConverter() {
}
/**
* App to flow row key is of the form clusterId!appId with the 2 segments
* separated by !. The sizes below indicate sizes of both of these segments
* in sequence. clusterId is a string. appId is represented as 12 bytes w.
* cluster Timestamp part of appid taking 8 bytes(long) and seq id taking 4
* bytes(int). Strings are variable in size (i.e. end whenever separator is
* encountered). This is used while decoding and helps in determining where
* to split.
*/
private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT };
/*
* (non-Javadoc)
*
* Encodes AppToFlowRowKey object into a byte array with each
* component/field in AppToFlowRowKey separated by Separator#QUALIFIERS.
* This leads to an app to flow table row key of the form clusterId!appId
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#encode(java.lang.Object)
*/
@Override
public byte[] encode(AppToFlowRowKey rowKey) {
byte[] first =
Separator.encode(rowKey.getClusterId(), Separator.SPACE,
Separator.TAB, Separator.QUALIFIERS);
byte[] second = appIDKeyConverter.encode(rowKey.getAppId());
return Separator.QUALIFIERS.join(first, second);
}
/*
* (non-Javadoc)
*
* Decodes an app to flow row key of the form clusterId!appId represented
* in byte format and converts it into an AppToFlowRowKey object.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#decode(byte[])
*/
@Override
public AppToFlowRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 2) {
throw new IllegalArgumentException("the row key is not valid for "
+ "the app-to-flow table");
}
String clusterId =
Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String appId = appIDKeyConverter.decode(rowKeyComponents[1]);
return new AppToFlowRowKey(clusterId, appId);
}
}
}

View File

@ -1,96 +0,0 @@
/**
* 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.apptoflow;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
/**
* Encodes and decodes row key for app_flow table.
* The row key is of the form : clusterId!appId.
* clusterId is a string and appId is encoded/decoded using
* {@link AppIdKeyConverter}.
*/
public final class AppToFlowRowKeyConverter
implements KeyConverter<AppToFlowRowKey> {
private static final AppToFlowRowKeyConverter INSTANCE =
new AppToFlowRowKeyConverter();
public static AppToFlowRowKeyConverter getInstance() {
return INSTANCE;
}
private AppToFlowRowKeyConverter() {
}
// App to flow row key is of the form clusterId!appId with the 2 segments
// separated by !. The sizes below indicate sizes of both of these segments
// in sequence. clusterId is a string. appId is represented as 12 bytes with
// cluster Timestamp part of appid being 8 bytes(long) and seq id being 4
// bytes(int).
// Strings are variable in size (i.e. end whenever separator is encountered).
// This is used while decoding and helps in determining where to split.
private static final int[] SEGMENT_SIZES = {
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT };
/*
* (non-Javadoc)
*
* Encodes AppToFlowRowKey object into a byte array with each component/field
* in AppToFlowRowKey separated by Separator#QUALIFIERS. This leads to an
* app to flow table row key of the form clusterId!appId
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #encode(java.lang.Object)
*/
@Override
public byte[] encode(AppToFlowRowKey rowKey) {
byte[] first = Separator.encode(rowKey.getClusterId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
byte[] second = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
return Separator.QUALIFIERS.join(first, second);
}
/*
* (non-Javadoc)
*
* Decodes an app to flow row key of the form clusterId!appId represented in
* byte format and converts it into an AppToFlowRowKey object.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #decode(byte[])
*/
@Override
public AppToFlowRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 2) {
throw new IllegalArgumentException("the row key is not valid for " +
"the app-to-flow table");
}
String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[1]);
return new AppToFlowRowKey(clusterId, appId);
}
}

View File

@ -28,13 +28,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
* (long - 8 bytes) followed by sequence id section of app id (int - 4 bytes).
*/
public final class AppIdKeyConverter implements KeyConverter<String> {
private static final AppIdKeyConverter INSTANCE = new AppIdKeyConverter();
public static AppIdKeyConverter getInstance() {
return INSTANCE;
}
private AppIdKeyConverter() {
public AppIdKeyConverter() {
}
/*
@ -58,7 +53,7 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
ApplicationId appId = ConverterUtils.toApplicationId(appIdStr);
byte[] appIdBytes = new byte[getKeySize()];
byte[] clusterTs = Bytes.toBytes(
TimelineStorageUtils.invertLong(appId.getClusterTimestamp()));
LongConverter.invertLong(appId.getClusterTimestamp()));
System.arraycopy(clusterTs, 0, appIdBytes, 0, Bytes.SIZEOF_LONG);
byte[] seqId = Bytes.toBytes(TimelineStorageUtils.invertInt(appId.getId()));
System.arraycopy(seqId, 0, appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT);
@ -83,7 +78,7 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
if (appIdBytes.length != getKeySize()) {
throw new IllegalArgumentException("Invalid app id in byte format");
}
long clusterTs = TimelineStorageUtils.invertLong(
long clusterTs = LongConverter.invertLong(
Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
int seqId = TimelineStorageUtils.invertInt(
Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));

View File

@ -26,6 +26,8 @@ public class EventColumnName {
private final String id;
private final Long timestamp;
private final String infoKey;
private final KeyConverter<EventColumnName> eventColumnNameConverter =
new EventColumnNameConverter();
public EventColumnName(String id, Long timestamp, String infoKey) {
this.id = id;
@ -45,4 +47,17 @@ public class EventColumnName {
return infoKey;
}
/**
* @return a byte array with each components/fields separated by
* Separator#VALUES. This leads to an event column name of the form
* eventId=timestamp=infokey. If both timestamp and infokey are null,
* then a qualifier of the form eventId=timestamp= is returned. If
* only infokey is null, then a qualifier of the form eventId= is
* returned. These prefix forms are useful for queries that intend to
* retrieve more than one specific column name.
*/
public byte[] getColumnQualifier() {
return eventColumnNameConverter.encode(this);
}
}

View File

@ -30,14 +30,8 @@ import org.apache.hadoop.hbase.util.Bytes;
*/
public final class EventColumnNameConverter
implements KeyConverter<EventColumnName> {
private static final EventColumnNameConverter INSTANCE =
new EventColumnNameConverter();
public static EventColumnNameConverter getInstance() {
return INSTANCE;
}
private EventColumnNameConverter() {
public EventColumnNameConverter() {
}
// eventId=timestamp=infokey are of types String, Long String
@ -69,7 +63,7 @@ public final class EventColumnNameConverter
return Separator.VALUES.join(first, Separator.EMPTY_BYTES);
}
byte[] second = Bytes.toBytes(
TimelineStorageUtils.invertLong(key.getTimestamp()));
LongConverter.invertLong(key.getTimestamp()));
if (key.getInfoKey() == null) {
return Separator.VALUES.join(first, second, Separator.EMPTY_BYTES);
}
@ -96,7 +90,7 @@ public final class EventColumnNameConverter
}
String id = Separator.decode(Bytes.toString(components[0]),
Separator.VALUES, Separator.TAB, Separator.SPACE);
Long ts = TimelineStorageUtils.invertLong(Bytes.toLong(components[1]));
Long ts = LongConverter.invertLong(Bytes.toLong(components[1]));
String infoKey = components[2].length == 0 ? null :
Separator.decode(Bytes.toString(components[2]),
Separator.VALUES, Separator.TAB, Separator.SPACE);

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.common;
import java.io.IOException;
import java.io.Serializable;
import org.apache.hadoop.hbase.util.Bytes;
@ -26,14 +27,15 @@ import org.apache.hadoop.hbase.util.Bytes;
* Encodes a value by interpreting it as a Long and converting it to bytes and
* decodes a set of bytes as a Long.
*/
public final class LongConverter implements NumericValueConverter {
private static final LongConverter INSTANCE = new LongConverter();
public final class LongConverter implements NumericValueConverter,
Serializable {
private LongConverter() {
}
/**
* Added because we implement Comparator<Number>.
*/
private static final long serialVersionUID = 1L;
public static LongConverter getInstance() {
return INSTANCE;
public LongConverter() {
}
@Override
@ -76,4 +78,17 @@ public final class LongConverter implements NumericValueConverter {
}
return sum;
}
/**
* Converts a timestamp into it's inverse timestamp to be used in (row) keys
* where we want to have the most recent timestamp in the top of the table
* (scans start at the most recent timestamp first).
*
* @param key value to be inverted so that the latest version will be first in
* a scan.
* @return inverted long
*/
public static long invertLong(long key) {
return Long.MAX_VALUE - key;
}
}

View File

@ -23,13 +23,13 @@ import java.io.IOException;
* Encodes and decodes column names / row keys which are long.
*/
public final class LongKeyConverter implements KeyConverter<Long> {
private static final LongKeyConverter INSTANCE = new LongKeyConverter();
public static LongKeyConverter getInstance() {
return INSTANCE;
}
/**
* To delegate the actual work to.
*/
private final LongConverter longConverter = new LongConverter();
private LongKeyConverter() {
public LongKeyConverter() {
}
/*
@ -44,7 +44,7 @@ public final class LongKeyConverter implements KeyConverter<Long> {
try {
// IOException will not be thrown here as we are explicitly passing
// Long.
return LongConverter.getInstance().encodeValue(key);
return longConverter.encodeValue(key);
} catch (IOException e) {
return null;
}
@ -60,7 +60,7 @@ public final class LongKeyConverter implements KeyConverter<Long> {
@Override
public Long decode(byte[] bytes) {
try {
return (Long) LongConverter.getInstance().decodeValue(bytes);
return (Long) longConverter.decodeValue(bytes);
} catch (IOException e) {
return null;
}

View File

@ -0,0 +1,42 @@
/**
* 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.common;
/**
* In queries where a single result is needed, an exact rowkey can be used
* through the corresponding rowkey#getRowKey() method. For queries that need to
* scan over a range of rowkeys, a partial (the initial part) of rowkeys are
* used. Classes implementing RowKeyPrefix indicate that they are the initial
* part of rowkeys, with different constructors with fewer number of argument to
* form a partial rowkey, a prefix.
*
* @param <R> indicating the type of rowkey that a particular implementation is
* a prefix for.
*/
public interface RowKeyPrefix<R> {
/**
* Create a row key prefix, meaning a partial rowkey that can be used in range
* scans. Which fields are included in the prefix will depend on the
* constructor of the specific instance that was used. Output depends on which
* constructor was used.
* @return a prefix of the following form {@code fist!second!...!last!}
*/
byte[] getRowKeyPrefix();
}

View File

@ -24,13 +24,8 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.common;
* added later, if required in the associated ColumnPrefix implementations.
*/
public final class StringKeyConverter implements KeyConverter<String> {
private static final StringKeyConverter INSTANCE = new StringKeyConverter();
public static StringKeyConverter getInstance() {
return INSTANCE;
}
private StringKeyConverter() {
public StringKeyConverter() {
}
/*

View File

@ -18,14 +18,12 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.common;
import java.io.IOException;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.SortedSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -37,13 +35,10 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
@ -52,7 +47,6 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilte
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
@ -72,19 +66,6 @@ public final class TimelineStorageUtils {
/** milliseconds in one day. */
public static final long MILLIS_ONE_DAY = 86400000L;
/**
* Converts a timestamp into it's inverse timestamp to be used in (row) keys
* where we want to have the most recent timestamp in the top of the table
* (scans start at the most recent timestamp first).
*
* @param key value to be inverted so that the latest version will be first in
* a scan.
* @return inverted long
*/
public static long invertLong(long key) {
return Long.MAX_VALUE - key;
}
/**
* Converts an int into it's inverse int to be used in (row) keys
* where we want to have the largest int value in the top of the table
@ -163,66 +144,6 @@ public final class TimelineStorageUtils {
return 0;
}
/**
* checks if an application has finished.
*
* @param te TimlineEntity object.
* @return true if application has finished else false
*/
public static boolean isApplicationFinished(TimelineEntity te) {
SortedSet<TimelineEvent> allEvents = te.getEvents();
if ((allEvents != null) && (allEvents.size() > 0)) {
TimelineEvent event = allEvents.last();
if (event.getId().equals(
ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) {
return true;
}
}
return false;
}
/**
* Check if we have a certain field amongst fields to retrieve. This method
* checks against {@link Field#ALL} as well because that would mean field
* passed needs to be matched.
*
* @param fieldsToRetrieve fields to be retrieved.
* @param requiredField fields to be checked in fieldsToRetrieve.
* @return true if has the required field, false otherwise.
*/
public static boolean hasField(EnumSet<Field> fieldsToRetrieve,
Field requiredField) {
return fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(requiredField);
}
/**
* Checks if the input TimelineEntity object is an ApplicationEntity.
*
* @param te TimelineEntity object.
* @return true if input is an ApplicationEntity, false otherwise
*/
public static boolean isApplicationEntity(TimelineEntity te) {
return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
}
/**
* @param te TimelineEntity object.
* @param eventId event with this id needs to be fetched
* @return TimelineEvent if TimelineEntity contains the desired event.
*/
public static TimelineEvent getApplicationEvent(TimelineEntity te,
String eventId) {
if (isApplicationEntity(te)) {
for (TimelineEvent event : te.getEvents()) {
if (event.getId().equals(eventId)) {
return event;
}
}
}
return null;
}
/**
* Returns the first seen aggregation operation as seen in the list of input
* tags or null otherwise.
@ -646,98 +567,6 @@ public final class TimelineStorageUtils {
return appId;
}
/**
* Helper method for reading relationship.
*
* @param <T> Describes the type of column prefix.
* @param entity entity to fill.
* @param result result from HBase.
* @param prefix column prefix.
* @param isRelatedTo if true, means relationship is to be added to
* isRelatedTo, otherwise its added to relatesTo.
* @throws IOException if any problem is encountered while reading result.
*/
public static <T> void readRelationship(
TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
boolean isRelatedTo) throws IOException {
// isRelatedTo and relatesTo are of type Map<String, Set<String>>
Map<String, Object> columns =
prefix.readResults(result, StringKeyConverter.getInstance());
for (Map.Entry<String, Object> column : columns.entrySet()) {
for (String id : Separator.VALUES.splitEncoded(
column.getValue().toString())) {
if (isRelatedTo) {
entity.addIsRelatedToEntity(column.getKey(), id);
} else {
entity.addRelatesToEntity(column.getKey(), id);
}
}
}
}
/**
* Helper method for reading key-value pairs for either info or config.
*
* @param <T> Describes the type of column prefix.
* @param entity entity to fill.
* @param result result from HBase.
* @param prefix column prefix.
* @param isConfig if true, means we are reading configs, otherwise info.
* @throws IOException if any problem is encountered while reading result.
*/
public static <T> void readKeyValuePairs(
TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
boolean isConfig) throws IOException {
// info and configuration are of type Map<String, Object or String>
Map<String, Object> columns =
prefix.readResults(result, StringKeyConverter.getInstance());
if (isConfig) {
for (Map.Entry<String, Object> column : columns.entrySet()) {
entity.addConfig(column.getKey(), column.getValue().toString());
}
} else {
entity.addInfo(columns);
}
}
/**
* Read events from the entity table or the application table. The column name
* is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted
* if there is no info associated with the event.
*
* @param <T> Describes the type of column prefix.
* @param entity entity to fill.
* @param result HBase Result.
* @param prefix column prefix.
* @throws IOException if any problem is encountered while reading result.
*/
public static <T> void readEvents(TimelineEntity entity, Result result,
ColumnPrefix<T> prefix) throws IOException {
Map<String, TimelineEvent> eventsMap = new HashMap<>();
Map<EventColumnName, Object> eventsResult =
prefix.readResults(result, EventColumnNameConverter.getInstance());
for (Map.Entry<EventColumnName, Object>
eventResult : eventsResult.entrySet()) {
EventColumnName eventColumnName = eventResult.getKey();
String key = eventColumnName.getId() +
Long.toString(eventColumnName.getTimestamp());
// Retrieve previously seen event to add to it
TimelineEvent event = eventsMap.get(key);
if (event == null) {
// First time we're seeing this event, add it to the eventsMap
event = new TimelineEvent();
event.setId(eventColumnName.getId());
event.setTimestamp(eventColumnName.getTimestamp());
eventsMap.put(key, event);
}
if (eventColumnName.getInfoKey() != null) {
event.addInfo(eventColumnName.getInfoKey(), eventResult.getValue());
}
}
Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
entity.addEvents(eventsSet);
}
public static boolean isFlowRunTable(HRegionInfo hRegionInfo,
Configuration conf) {
String regionTableName = hRegionInfo.getTable().getNameAsString();

View File

@ -49,8 +49,7 @@ public enum EntityColumn implements Column<EntityTable> {
/**
* When the entity was created.
*/
CREATED_TIME(EntityColumnFamily.INFO, "created_time",
LongConverter.getInstance()),
CREATED_TIME(EntityColumnFamily.INFO, "created_time", new LongConverter()),
/**
* The version of the flow that this entity belongs to.

View File

@ -67,8 +67,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
/**
* Metrics are stored with the metric name as the column name.
*/
METRIC(EntityColumnFamily.METRICS, null,
LongConverter.getInstance());
METRIC(EntityColumnFamily.METRICS, null, new LongConverter());
private final ColumnHelper<EntityTable> column;
private final ColumnFamily<EntityTable> columnFamily;

View File

@ -17,6 +17,12 @@
*/
package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
/**
* Represents a rowkey for the entity table.
*/
@ -28,6 +34,8 @@ public class EntityRowKey {
private final String appId;
private final String entityType;
private final String entityId;
private final KeyConverter<EntityRowKey> entityRowKeyConverter =
new EntityRowKeyConverter();
public EntityRowKey(String clusterId, String userId, String flowName,
Long flowRunId, String appId, String entityType, String entityId) {
@ -68,62 +76,15 @@ public class EntityRowKey {
return entityId;
}
/**
* Constructs a row key prefix for the entity table as follows:
* {@code userName!clusterId!flowName!flowRunId!AppId}.
*
* @param clusterId Context cluster id.
* @param userId User name.
* @param flowName Flow name.
* @param flowRunId Run Id for the flow.
* @param appId Application Id.
* @return byte array with the row key prefix.
*/
public static byte[] getRowKeyPrefix(String clusterId, String userId,
String flowName, Long flowRunId, String appId) {
return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
clusterId, userId, flowName, flowRunId, appId, null, null));
}
/**
* Constructs a row key prefix for the entity table as follows:
* {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}.
* Typically used while querying multiple entities of a particular entity
* type.
*
* @param clusterId Context cluster id.
* @param userId User name.
* @param flowName Flow name.
* @param flowRunId Run Id for the flow.
* @param appId Application Id.
* @param entityType Entity type.
* @return byte array with the row key prefix.
*/
public static byte[] getRowKeyPrefix(String clusterId, String userId,
String flowName, Long flowRunId, String appId, String entityType) {
return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
clusterId, userId, flowName, flowRunId, appId, entityType, null));
}
/**
* Constructs a row key for the entity table as follows:
* {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}.
* Typically used while querying a specific entity.
*
* @param clusterId Context cluster id.
* @param userId User name.
* @param flowName Flow name.
* @param flowRunId Run Id for the flow.
* @param appId Application Id.
* @param entityType Entity type.
* @param entityId Entity Id.
* @return byte array with the row key.
*/
public static byte[] getRowKey(String clusterId, String userId,
String flowName, Long flowRunId, String appId, String entityType,
String entityId) {
return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
clusterId, userId, flowName, flowRunId, appId, entityType, entityId));
public byte[] getRowKey() {
return entityRowKeyConverter.encode(this);
}
/**
@ -133,6 +94,132 @@ public class EntityRowKey {
* @return An <cite>EntityRowKey</cite> object.
*/
public static EntityRowKey parseRowKey(byte[] rowKey) {
return EntityRowKeyConverter.getInstance().decode(rowKey);
return new EntityRowKeyConverter().decode(rowKey);
}
/**
* Encodes and decodes row key for entity table. The row key is of the form :
* userName!clusterId!flowName!flowRunId!appId!entityType!entityId. flowRunId
* is a long, appId is encoded/decoded using {@link AppIdKeyConverter} and
* rest are strings.
* <p>
*/
final private static class EntityRowKeyConverter implements
KeyConverter<EntityRowKey> {
private final AppIdKeyConverter appIDKeyConverter = new AppIdKeyConverter();
private EntityRowKeyConverter() {
}
/**
* Entity row key is of the form
* userName!clusterId!flowName!flowRunId!appId!entityType!entityId w. each
* segment separated by !. The sizes below indicate sizes of each one of
* these segments in sequence. clusterId, userName, flowName, entityType and
* entityId are strings. flowrunId is a long hence 8 bytes in size. app id
* is represented as 12 bytes with cluster timestamp part of appid being 8
* bytes (long) and seq id being 4 bytes(int). Strings are variable in size
* (i.e. end whenever separator is encountered). This is used while decoding
* and helps in determining where to split.
*/
private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE };
/*
* (non-Javadoc)
*
* Encodes EntityRowKey object into a byte array with each component/field
* in EntityRowKey separated by Separator#QUALIFIERS. This leads to an
* entity table row key of the form
* userName!clusterId!flowName!flowRunId!appId!entityType!entityId If
* entityType in passed EntityRowKey object is null (and the fields
* preceding it i.e. clusterId, userId and flowName, flowRunId and appId
* are not null), this returns a row key prefix of the form
* userName!clusterId!flowName!flowRunId!appId! and if entityId in
* EntityRowKey is null (other 6 components are not null), this returns a
* row key prefix of the form
* userName!clusterId!flowName!flowRunId!appId!entityType! flowRunId is
* inverted while encoding as it helps maintain a descending order for row
* keys in entity table.
*
* @see org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#encode(java.lang.Object)
*/
@Override
public byte[] encode(EntityRowKey rowKey) {
byte[] user =
Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS);
byte[] cluster =
Separator.encode(rowKey.getClusterId(), Separator.SPACE,
Separator.TAB, Separator.QUALIFIERS);
byte[] flow =
Separator.encode(rowKey.getFlowName(), Separator.SPACE,
Separator.TAB, Separator.QUALIFIERS);
byte[] first = Separator.QUALIFIERS.join(user, cluster, flow);
// Note that flowRunId is a long, so we can't encode them all at the same
// time.
byte[] second =
Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId()));
byte[] third = appIDKeyConverter.encode(rowKey.getAppId());
if (rowKey.getEntityType() == null) {
return Separator.QUALIFIERS.join(first, second, third,
Separator.EMPTY_BYTES);
}
byte[] entityType =
Separator.encode(rowKey.getEntityType(), Separator.SPACE,
Separator.TAB, Separator.QUALIFIERS);
byte[] entityId =
rowKey.getEntityId() == null ? Separator.EMPTY_BYTES : Separator
.encode(rowKey.getEntityId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS);
byte[] fourth = Separator.QUALIFIERS.join(entityType, entityId);
return Separator.QUALIFIERS.join(first, second, third, fourth);
}
/*
* (non-Javadoc)
*
* Decodes an application row key of the form
* userName!clusterId!flowName!flowRunId!appId!entityType!entityId
* represented in byte format and converts it into an EntityRowKey object.
* flowRunId is inverted while decoding as it was inverted while encoding.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#decode(byte[])
*/
@Override
public EntityRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 7) {
throw new IllegalArgumentException("the row key is not valid for "
+ "an entity");
}
String userId =
Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String clusterId =
Separator.decode(Bytes.toString(rowKeyComponents[1]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String flowName =
Separator.decode(Bytes.toString(rowKeyComponents[2]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
Long flowRunId =
LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3]));
String appId = appIDKeyConverter.decode(rowKeyComponents[4]);
String entityType =
Separator.decode(Bytes.toString(rowKeyComponents[5]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String entityId =
Separator.decode(Bytes.toString(rowKeyComponents[6]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
entityType, entityId);
}
}
}

View File

@ -1,143 +0,0 @@
/**
* 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.entity;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
/**
* Encodes and decodes row key for entity table.
* The row key is of the form :
* userName!clusterId!flowName!flowRunId!appId!entityType!entityId.
* flowRunId is a long, appId is encoded/decoded using
* {@link AppIdKeyConverter} and rest are strings.
*/
public final class EntityRowKeyConverter implements KeyConverter<EntityRowKey> {
private static final EntityRowKeyConverter INSTANCE =
new EntityRowKeyConverter();
public static EntityRowKeyConverter getInstance() {
return INSTANCE;
}
private EntityRowKeyConverter() {
}
// Entity row key is of the form
// userName!clusterId!flowName!flowRunId!appId!entityType!entityId with each
// segment separated by !. The sizes below indicate sizes of each one of these
// segements in sequence. clusterId, userName, flowName, entityType and
// entityId are strings. flowrunId is a long hence 8 bytes in size. app id is
// represented as 12 bytes with cluster timestamp part of appid being 8 bytes
// (long) and seq id being 4 bytes(int).
// Strings are variable in size (i.e. end whenever separator is encountered).
// This is used while decoding and helps in determining where to split.
private static final int[] SEGMENT_SIZES = {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Bytes.SIZEOF_LONG, AppIdKeyConverter.getKeySize(),
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE };
/*
* (non-Javadoc)
*
* Encodes EntityRowKey object into a byte array with each component/field in
* EntityRowKey separated by Separator#QUALIFIERS. This leads to an entity
* table row key of the form
* userName!clusterId!flowName!flowRunId!appId!entityType!entityId
* If entityType in passed EntityRowKey object is null (and the fields
* preceding it i.e. clusterId, userId and flowName, flowRunId and appId are
* not null), this returns a row key prefix of the form
* userName!clusterId!flowName!flowRunId!appId! and if entityId in
* EntityRowKey is null (other 6 components are not null), this returns a row
* key prefix of the form
* userName!clusterId!flowName!flowRunId!appId!entityType!
* flowRunId is inverted while encoding as it helps maintain a descending
* order for row keys in entity table.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #encode(java.lang.Object)
*/
@Override
public byte[] encode(EntityRowKey rowKey) {
byte[] user = Separator.encode(rowKey.getUserId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
byte[] cluster = Separator.encode(rowKey.getClusterId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
byte[] flow = Separator.encode(rowKey.getFlowName(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
byte[] first = Separator.QUALIFIERS.join(user, cluster, flow);
// Note that flowRunId is a long, so we can't encode them all at the same
// time.
byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(
rowKey.getFlowRunId()));
byte[] third = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
if (rowKey.getEntityType() == null) {
return Separator.QUALIFIERS.join(
first, second, third, Separator.EMPTY_BYTES);
}
byte[] entityType = Separator.encode(rowKey.getEntityType(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
byte[] entityId = rowKey.getEntityId() == null ? Separator.EMPTY_BYTES :
Separator.encode(rowKey.getEntityId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS);
byte[] fourth = Separator.QUALIFIERS.join(entityType, entityId);
return Separator.QUALIFIERS.join(first, second, third, fourth);
}
/*
* (non-Javadoc)
*
* Decodes an application row key of the form
* userName!clusterId!flowName!flowRunId!appId!entityType!entityId represented
* in byte format and converts it into an EntityRowKey object. flowRunId is
* inverted while decoding as it was inverted while encoding.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #decode(byte[])
*/
@Override
public EntityRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 7) {
throw new IllegalArgumentException("the row key is not valid for " +
"an entity");
}
String userId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
Long flowRunId =
TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[4]);
String entityType = Separator.decode(Bytes.toString(rowKeyComponents[5]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String entityId =Separator.decode(Bytes.toString(rowKeyComponents[6]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
entityType, entityId);
}
}

View File

@ -0,0 +1,74 @@
/**
* 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.entity;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
/**
* Represents a partial rowkey without the entityId or without entityType and
* entityId for the entity table.
*
*/
public class EntityRowKeyPrefix extends EntityRowKey implements
RowKeyPrefix<EntityRowKey> {
/**
* Creates a prefix which generates the following rowKeyPrefixes for the
* entity table:
* {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}.
*
* @param clusterId identifying the cluster
* @param userId identifying the user
* @param flowName identifying the flow
* @param flowRunId identifying the individual run of this flow
* @param appId identifying the application
* @param entityType which entity type
*/
public EntityRowKeyPrefix(String clusterId, String userId, String flowName,
Long flowRunId, String appId, String entityType) {
super(clusterId, userId, flowName, flowRunId, appId, entityType, null);
}
/**
* Creates a prefix which generates the following rowKeyPrefixes for the
* entity table:
* {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}.
*
* @param clusterId identifying the cluster
* @param userId identifying the user
* @param flowName identifying the flow
* @param flowRunId identifying the individual run of this flow
* @param appId identifying the application
*/
public EntityRowKeyPrefix(String clusterId, String userId, String flowName,
Long flowRunId, String appId) {
super(clusterId, userId, flowName, flowRunId, appId, null, null);
}
/*
* (non-Javadoc)
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.application.
* RowKeyPrefix#getRowKeyPrefix()
*/
public byte[] getRowKeyPrefix() {
return super.getRowKey();
}
}

View File

@ -17,6 +17,10 @@
*/
package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
/**
@ -28,11 +32,37 @@ public class FlowActivityRowKey {
private final Long dayTs;
private final String userId;
private final String flowName;
private final KeyConverter<FlowActivityRowKey> flowActivityRowKeyConverter =
new FlowActivityRowKeyConverter();
/**
* @param clusterId identifying the cluster
* @param dayTs to be converted to the top of the day timestamp
* @param userId identifying user
* @param flowName identifying the flow
*/
public FlowActivityRowKey(String clusterId, Long dayTs, String userId,
String flowName) {
this(clusterId, dayTs, userId, flowName, true);
}
/**
* @param clusterId identifying the cluster
* @param timestamp when the flow activity happened. May be converted to the
* top of the day depending on the convertDayTsToTopOfDay argument.
* @param userId identifying user
* @param flowName identifying the flow
* @param convertDayTsToTopOfDay if true and timestamp isn't null, then
* timestamp will be converted to the top-of-the day timestamp
*/
protected FlowActivityRowKey(String clusterId, Long timestamp, String userId,
String flowName, boolean convertDayTsToTopOfDay) {
this.clusterId = clusterId;
this.dayTs = dayTs;
if (convertDayTsToTopOfDay && (timestamp != null)) {
this.dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(timestamp);
} else {
this.dayTs = timestamp;
}
this.userId = userId;
this.flowName = flowName;
}
@ -53,47 +83,14 @@ public class FlowActivityRowKey {
return flowName;
}
/**
* Constructs a row key prefix for the flow activity table as follows:
* {@code clusterId!}.
*
* @param clusterId Cluster Id.
* @return byte array with the row key prefix
*/
public static byte[] getRowKeyPrefix(String clusterId) {
return FlowActivityRowKeyConverter.getInstance().encode(
new FlowActivityRowKey(clusterId, null, null, null));
}
/**
* Constructs a row key prefix for the flow activity table as follows:
* {@code clusterId!dayTimestamp!}.
*
* @param clusterId Cluster Id.
* @param dayTs Start of the day timestamp.
* @return byte array with the row key prefix
*/
public static byte[] getRowKeyPrefix(String clusterId, long dayTs) {
return FlowActivityRowKeyConverter.getInstance().encode(
new FlowActivityRowKey(clusterId, dayTs, null, null));
}
/**
* Constructs a row key for the flow activity table as follows:
* {@code clusterId!dayTimestamp!user!flowName}.
*
* @param clusterId Cluster Id.
* @param eventTs event's TimeStamp.
* @param userId User Id.
* @param flowName Flow Name.
* @return byte array for the row key
*/
public static byte[] getRowKey(String clusterId, long eventTs, String userId,
String flowName) {
// convert it to Day's time stamp
eventTs = TimelineStorageUtils.getTopOfTheDayTimestamp(eventTs);
return FlowActivityRowKeyConverter.getInstance().encode(
new FlowActivityRowKey(clusterId, eventTs, userId, flowName));
public byte[] getRowKey() {
return flowActivityRowKeyConverter.encode(this);
}
/**
@ -103,6 +100,97 @@ public class FlowActivityRowKey {
* @return A <cite>FlowActivityRowKey</cite> object.
*/
public static FlowActivityRowKey parseRowKey(byte[] rowKey) {
return FlowActivityRowKeyConverter.getInstance().decode(rowKey);
return new FlowActivityRowKeyConverter().decode(rowKey);
}
/**
* Encodes and decodes row key for flow activity table. The row key is of the
* form : clusterId!dayTimestamp!user!flowName. dayTimestamp(top of the day
* timestamp) is a long and rest are strings.
* <p>
*/
final private static class FlowActivityRowKeyConverter implements
KeyConverter<FlowActivityRowKey> {
private FlowActivityRowKeyConverter() {
}
/**
* The flow activity row key is of the form
* clusterId!dayTimestamp!user!flowName with each segment separated by !.
* The sizes below indicate sizes of each one of these segements in
* sequence. clusterId, user and flowName are strings. Top of the day
* timestamp is a long hence 8 bytes in size. Strings are variable in size
* (i.e. they end whenever separator is encountered). This is used while
* decoding and helps in determining where to split.
*/
private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE };
/*
* (non-Javadoc)
*
* Encodes FlowActivityRowKey object into a byte array with each
* component/field in FlowActivityRowKey separated by Separator#QUALIFIERS.
* This leads to an flow activity table row key of the form
* clusterId!dayTimestamp!user!flowName. If dayTimestamp in passed
* FlowActivityRowKey object is null and clusterId is not null, then this
* returns a row key prefix as clusterId! and if userId in
* FlowActivityRowKey is null (and the fields preceding it i.e. clusterId
* and dayTimestamp are not null), this returns a row key prefix as
* clusterId!dayTimeStamp! dayTimestamp is inverted while encoding as it
* helps maintain a descending order for row keys in flow activity table.
*
* @see org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#encode(java.lang.Object)
*/
@Override
public byte[] encode(FlowActivityRowKey rowKey) {
if (rowKey.getDayTimestamp() == null) {
return Separator.QUALIFIERS.join(Separator.encode(
rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS), Separator.EMPTY_BYTES);
}
if (rowKey.getUserId() == null) {
return Separator.QUALIFIERS.join(Separator.encode(
rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS), Bytes.toBytes(LongConverter
.invertLong(rowKey.getDayTimestamp())), Separator.EMPTY_BYTES);
}
return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Bytes
.toBytes(LongConverter.invertLong(rowKey.getDayTimestamp())),
Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS));
}
/*
* (non-Javadoc)
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#decode(byte[])
*/
@Override
public FlowActivityRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 4) {
throw new IllegalArgumentException("the row key is not valid for "
+ "a flow activity");
}
String clusterId =
Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
Long dayTs = LongConverter.invertLong(Bytes.toLong(rowKeyComponents[1]));
String userId =
Separator.decode(Bytes.toString(rowKeyComponents[2]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String flowName =
Separator.decode(Bytes.toString(rowKeyComponents[3]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
return new FlowActivityRowKey(clusterId, dayTs, userId, flowName);
}
}
}

View File

@ -1,115 +0,0 @@
/**
* 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.flow;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
/**
* Encodes and decodes row key for flow activity table.
* The row key is of the form : clusterId!dayTimestamp!user!flowName.
* dayTimestamp(top of the day timestamp) is a long and rest are strings.
*/
public final class FlowActivityRowKeyConverter implements
KeyConverter<FlowActivityRowKey> {
private static final FlowActivityRowKeyConverter INSTANCE =
new FlowActivityRowKeyConverter();
public static FlowActivityRowKeyConverter getInstance() {
return INSTANCE;
}
private FlowActivityRowKeyConverter() {
}
// Flow activity row key is of the form clusterId!dayTimestamp!user!flowName
// with each segment separated by !. The sizes below indicate sizes of each
// one of these segements in sequence. clusterId, user and flowName are
// strings. Top of the day timestamp is a long hence 8 bytes in size.
// Strings are variable in size (i.e. end whenever separator is encountered).
// This is used while decoding and helps in determining where to split.
private static final int[] SEGMENT_SIZES = {
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE };
/*
* (non-Javadoc)
*
* Encodes FlowActivityRowKey object into a byte array with each
* component/field in FlowActivityRowKey separated by Separator#QUALIFIERS.
* This leads to an flow activity table row key of the form
* clusterId!dayTimestamp!user!flowName
* If dayTimestamp in passed FlowActivityRowKey object is null and clusterId
* is not null, this returns a row key prefix as clusterId! and if userId in
* FlowActivityRowKey is null (and the fields preceding it i.e. clusterId and
* dayTimestamp are not null), this returns a row key prefix as
* clusterId!dayTimeStamp!
* dayTimestamp is inverted while encoding as it helps maintain a descending
* order for row keys in flow activity table.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #encode(java.lang.Object)
*/
@Override
public byte[] encode(FlowActivityRowKey rowKey) {
if (rowKey.getDayTimestamp() == null) {
return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS),
Separator.EMPTY_BYTES);
}
if (rowKey.getUserId() == null) {
return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS),
Bytes.toBytes(TimelineStorageUtils.invertLong(
rowKey.getDayTimestamp())), Separator.EMPTY_BYTES);
}
return Separator.QUALIFIERS.join(
Separator.encode(rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS),
Bytes.toBytes(
TimelineStorageUtils.invertLong(rowKey.getDayTimestamp())),
Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS),
Separator.encode(rowKey.getFlowName(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS));
}
@Override
public FlowActivityRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 4) {
throw new IllegalArgumentException("the row key is not valid for "
+ "a flow activity");
}
String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
Long dayTs =
TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[1]));
String userId = Separator.decode(Bytes.toString(rowKeyComponents[2]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String flowName = Separator.decode(Bytes.toString(rowKeyComponents[3]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
return new FlowActivityRowKey(clusterId, dayTs, userId, flowName);
}
}

View File

@ -0,0 +1,60 @@
/**
* 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.flow;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
/**
* A prefix partial rowkey for flow activities.
*/
public class FlowActivityRowKeyPrefix extends FlowActivityRowKey implements
RowKeyPrefix<FlowActivityRowKey> {
/**
* Constructs a row key prefix for the flow activity table as follows:
* {@code clusterId!dayTimestamp!}.
*
* @param clusterId Cluster Id.
* @param dayTs Start of the day timestamp.
*/
public FlowActivityRowKeyPrefix(String clusterId, Long dayTs) {
super(clusterId, dayTs, null, null, false);
}
/**
* Constructs a row key prefix for the flow activity table as follows:
* {@code clusterId!}.
*
* @param clusterId identifying the cluster
*/
public FlowActivityRowKeyPrefix(String clusterId) {
super(clusterId, null, null, null, false);
}
/*
* (non-Javadoc)
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.application.
* RowKeyPrefix#getRowKeyPrefix()
*/
public byte[] getRowKeyPrefix() {
return super.getRowKey();
}
}

View File

@ -25,10 +25,10 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
/**
@ -41,14 +41,14 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
* application start times.
*/
MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time",
AggregationOperation.GLOBAL_MIN, LongConverter.getInstance()),
AggregationOperation.GLOBAL_MIN, new LongConverter()),
/**
* When the flow ended. This is the maximum of currently known application end
* times.
*/
MAX_END_TIME(FlowRunColumnFamily.INFO, "max_end_time",
AggregationOperation.GLOBAL_MAX, LongConverter.getInstance()),
AggregationOperation.GLOBAL_MAX, new LongConverter()),
/**
* The version of the flow that this flow belongs to.

View File

@ -41,7 +41,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
/**
* To store flow run info values.
*/
METRIC(FlowRunColumnFamily.INFO, "m", null, LongConverter.getInstance());
METRIC(FlowRunColumnFamily.INFO, "m", null, new LongConverter());
private final ColumnHelper<FlowRunTable> column;
private final ColumnFamily<FlowRunTable> columnFamily;

View File

@ -17,6 +17,11 @@
*/
package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
/**
* Represents a rowkey for the flow run table.
*/
@ -25,6 +30,8 @@ public class FlowRunRowKey {
private final String userId;
private final String flowName;
private final Long flowRunId;
private final FlowRunRowKeyConverter flowRunRowKeyConverter =
new FlowRunRowKeyConverter();
public FlowRunRowKey(String clusterId, String userId, String flowName,
Long flowRunId) {
@ -50,37 +57,17 @@ public class FlowRunRowKey {
return flowRunId;
}
/**
* Constructs a row key prefix for the flow run table as follows: {
* clusterId!userI!flowName!}.
*
* @param clusterId Cluster Id.
* @param userId User Id.
* @param flowName Flow Name.
* @return byte array with the row key prefix
*/
public static byte[] getRowKeyPrefix(String clusterId, String userId,
String flowName) {
return FlowRunRowKeyConverter.getInstance().encode(new FlowRunRowKey(
clusterId, userId, flowName, null));
}
/**
* Constructs a row key for the entity table as follows: {
* clusterId!userId!flowName!Inverted Flow Run Id}.
*
* @param clusterId Cluster Id.
* @param userId User Id.
* @param flowName Flow Name.
* @param flowRunId Run Id for the flow name.
* @return byte array with the row key
*/
public static byte[] getRowKey(String clusterId, String userId,
String flowName, Long flowRunId) {
return FlowRunRowKeyConverter.getInstance().encode(new FlowRunRowKey(
clusterId, userId, flowName, flowRunId));
public byte[] getRowKey() {
return flowRunRowKeyConverter.encode(this);
}
/**
* Given the raw row key as bytes, returns the row key as an object.
*
@ -88,7 +75,7 @@ public class FlowRunRowKey {
* @return A <cite>FlowRunRowKey</cite> object.
*/
public static FlowRunRowKey parseRowKey(byte[] rowKey) {
return FlowRunRowKeyConverter.getInstance().decode(rowKey);
return new FlowRunRowKeyConverter().decode(rowKey);
}
/**
@ -106,4 +93,98 @@ public class FlowRunRowKey {
flowKeyStr.append("}");
return flowKeyStr.toString();
}
/**
* Encodes and decodes row key for flow run table.
* The row key is of the form : clusterId!userId!flowName!flowrunId.
* flowrunId is a long and rest are strings.
* <p>
*/
final private static class FlowRunRowKeyConverter implements
KeyConverter<FlowRunRowKey> {
private FlowRunRowKeyConverter() {
}
/**
* The flow run row key is of the form clusterId!userId!flowName!flowrunId
* with each segment separated by !. The sizes below indicate sizes of each
* one of these segments in sequence. clusterId, userId and flowName are
* strings. flowrunId is a long hence 8 bytes in size. Strings are variable
* in size (i.e. end whenever separator is encountered). This is used while
* decoding and helps in determining where to split.
*/
private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG };
/*
* (non-Javadoc)
*
* Encodes FlowRunRowKey object into a byte array with each component/field
* in FlowRunRowKey separated by Separator#QUALIFIERS. This leads to an flow
* run row key of the form clusterId!userId!flowName!flowrunId If flowRunId
* in passed FlowRunRowKey object is null (and the fields preceding it i.e.
* clusterId, userId and flowName are not null), this returns a row key
* prefix of the form clusterId!userName!flowName! flowRunId is inverted
* while encoding as it helps maintain a descending order for flow keys in
* flow run table.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#encode(java.lang.Object)
*/
@Override
public byte[] encode(FlowRunRowKey rowKey) {
byte[] first =
Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Separator
.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(),
Separator.SPACE, Separator.TAB, Separator.QUALIFIERS));
if (rowKey.getFlowRunId() == null) {
return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
} else {
// Note that flowRunId is a long, so we can't encode them all at the
// same
// time.
byte[] second =
Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId()));
return Separator.QUALIFIERS.join(first, second);
}
}
/*
* (non-Javadoc)
*
* Decodes an flow run row key of the form
* clusterId!userId!flowName!flowrunId represented in byte format and
* converts it into an FlowRunRowKey object. flowRunId is inverted while
* decoding as it was inverted while encoding.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common
* .KeyConverter#decode(byte[])
*/
@Override
public FlowRunRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 4) {
throw new IllegalArgumentException("the row key is not valid for "
+ "a flow run");
}
String clusterId =
Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String userId =
Separator.decode(Bytes.toString(rowKeyComponents[1]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String flowName =
Separator.decode(Bytes.toString(rowKeyComponents[2]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
Long flowRunId =
LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3]));
return new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
}
}
}

View File

@ -1,120 +0,0 @@
/**
* 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.flow;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
/**
* Encodes and decodes row key for flow run table.
* The row key is of the form : clusterId!userId!flowName!flowrunId.
* flowrunId is a long and rest are strings.
*/
public final class FlowRunRowKeyConverter implements
KeyConverter<FlowRunRowKey> {
private static final FlowRunRowKeyConverter INSTANCE =
new FlowRunRowKeyConverter();
public static FlowRunRowKeyConverter getInstance() {
return INSTANCE;
}
private FlowRunRowKeyConverter() {
}
// Flow run row key is of the form
// clusterId!userId!flowName!flowrunId with each segment separated by !.
// The sizes below indicate sizes of each one of these segments in sequence.
// clusterId, userId and flowName are strings. flowrunId is a long hence 8
// bytes in size. Strings are variable in size (i.e. end whenever separator is
// encountered). This is used while decoding and helps in determining where to
// split.
private static final int[] SEGMENT_SIZES = {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Bytes.SIZEOF_LONG };
/*
* (non-Javadoc)
*
* Encodes FlowRunRowKey object into a byte array with each component/field in
* FlowRunRowKey separated by Separator#QUALIFIERS. This leads to an
* flow run row key of the form clusterId!userId!flowName!flowrunId
* If flowRunId in passed FlowRunRowKey object is null (and the fields
* preceding it i.e. clusterId, userId and flowName are not null), this
* returns a row key prefix of the form clusterId!userName!flowName!
* flowRunId is inverted while encoding as it helps maintain a descending
* order for flow keys in flow run table.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #encode(java.lang.Object)
*/
@Override
public byte[] encode(FlowRunRowKey rowKey) {
byte[] first = Separator.QUALIFIERS.join(
Separator.encode(rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS),
Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS),
Separator.encode(rowKey.getFlowName(), Separator.SPACE, Separator.TAB,
Separator.QUALIFIERS));
if (rowKey.getFlowRunId() == null) {
return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
} else {
// Note that flowRunId is a long, so we can't encode them all at the same
// time.
byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(
rowKey.getFlowRunId()));
return Separator.QUALIFIERS.join(first, second);
}
}
/*
* (non-Javadoc)
*
* Decodes an flow run row key of the form
* clusterId!userId!flowName!flowrunId represented in byte format and converts
* it into an FlowRunRowKey object. flowRunId is inverted while decoding as
* it was inverted while encoding.
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
* #decode(byte[])
*/
@Override
public FlowRunRowKey decode(byte[] rowKey) {
byte[][] rowKeyComponents =
Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
if (rowKeyComponents.length != 4) {
throw new IllegalArgumentException("the row key is not valid for " +
"a flow run");
}
String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String userId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
Long flowRunId =
TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
return new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
}
}

View File

@ -0,0 +1,54 @@
/**
* 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.flow;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
/**
* Represents a partial rowkey (without the flowRunId) for the flow run table.
*/
public class FlowRunRowKeyPrefix extends FlowRunRowKey implements
RowKeyPrefix<FlowRunRowKey> {
/**
* Constructs a row key prefix for the flow run table as follows:
* {@code clusterId!userI!flowName!}.
*
* @param clusterId identifying the cluster
* @param userId identifying the user
* @param flowName identifying the flow
*/
public FlowRunRowKeyPrefix(String clusterId, String userId,
String flowName) {
super(clusterId, userId, flowName, null);
}
/*
* (non-Javadoc)
*
* @see
* org.apache.hadoop.yarn.server.timelineservice.storage.application.
* RowKeyPrefix#getRowKeyPrefix()
*/
public byte[] getRowKeyPrefix() {
// We know we're a FlowRunRowKey with null florRunId, so we can simply
// delegate
return super.getRowKey();
}
}

View File

@ -46,8 +46,11 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import com.google.common.base.Preconditions;
@ -150,13 +153,13 @@ class ApplicationEntityReader extends GenericEntityReader {
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
// If INFO field has to be retrieved, add a filter for fetching columns
// with INFO column prefix.
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
if (hasField(fieldsToRetrieve, Field.INFO)) {
infoFamilyColsFilter.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.EQUAL, ApplicationColumnPrefix.INFO));
}
TimelineFilterList relatesTo = getFilters().getRelatesTo();
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
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(
@ -169,12 +172,11 @@ class ApplicationEntityReader extends GenericEntityReader {
// matched after fetching rows from HBase.
Set<String> relatesToCols =
TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo);
infoFamilyColsFilter.addFilter(
TimelineFilterUtils.createFiltersFromColumnQualifiers(
ApplicationColumnPrefix.RELATES_TO, relatesToCols));
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
ApplicationColumnPrefix.RELATES_TO, relatesToCols));
}
TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo();
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
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(
@ -187,12 +189,11 @@ class ApplicationEntityReader extends GenericEntityReader {
// matched after fetching rows from HBase.
Set<String> isRelatedToCols =
TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo);
infoFamilyColsFilter.addFilter(
TimelineFilterUtils.createFiltersFromColumnQualifiers(
ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols));
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols));
}
TimelineFilterList eventFilters = getFilters().getEventFilters();
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
if (hasField(fieldsToRetrieve, Field.EVENTS)) {
// If EVENTS field has to be retrieved, add a filter for fetching columns
// with EVENT column prefix.
infoFamilyColsFilter.addFilter(
@ -205,9 +206,8 @@ class ApplicationEntityReader extends GenericEntityReader {
// fetching rows from HBase.
Set<String> eventCols =
TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters);
infoFamilyColsFilter.addFilter(
TimelineFilterUtils.createFiltersFromColumnQualifiers(
ApplicationColumnPrefix.EVENT, eventCols));
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
ApplicationColumnPrefix.EVENT, eventCols));
}
return infoFamilyColsFilter;
}
@ -222,25 +222,25 @@ class ApplicationEntityReader extends GenericEntityReader {
private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) {
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
// Events not required.
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
infoColFamilyList.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.NOT_EQUAL, ApplicationColumnPrefix.EVENT));
}
// info not required.
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
if (!hasField(fieldsToRetrieve, Field.INFO)) {
infoColFamilyList.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.NOT_EQUAL, ApplicationColumnPrefix.INFO));
}
// is related to not required.
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
infoColFamilyList.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.NOT_EQUAL, ApplicationColumnPrefix.IS_RELATED_TO));
}
// relates to not required.
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
infoColFamilyList.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.NOT_EQUAL, ApplicationColumnPrefix.RELATES_TO));
@ -308,9 +308,10 @@ class ApplicationEntityReader extends GenericEntityReader {
protected Result getResult(Configuration hbaseConf, Connection conn,
FilterList filterList) throws IOException {
TimelineReaderContext context = getContext();
byte[] rowKey =
ApplicationRowKey.getRowKey(context.getClusterId(), context.getUserId(),
ApplicationRowKey applicationRowKey =
new ApplicationRowKey(context.getClusterId(), context.getUserId(),
context.getFlowName(), context.getFlowRunId(), context.getAppId());
byte[] rowKey = applicationRowKey.getRowKey();
Get get = new Get(rowKey);
get.setMaxVersions(getDataToRetrieve().getMetricsLimit());
if (filterList != null && !filterList.getFilters().isEmpty()) {
@ -345,10 +346,13 @@ class ApplicationEntityReader extends GenericEntityReader {
TimelineReaderContext context = getContext();
if (isSingleEntityRead()) {
// Get flow context information from AppToFlow table.
if (context.getFlowName() == null || context.getFlowRunId() == null ||
context.getUserId() == null) {
FlowContext flowContext = lookupFlowContext(
context.getClusterId(), context.getAppId(), hbaseConf, conn);
if (context.getFlowName() == null || context.getFlowRunId() == null
|| context.getUserId() == null) {
AppToFlowRowKey appToFlowRowKey =
new AppToFlowRowKey(context.getClusterId(), context.getAppId());
FlowContext flowContext =
lookupFlowContext(appToFlowRowKey,
hbaseConf, conn);
context.setFlowName(flowContext.getFlowName());
context.setFlowRunId(flowContext.getFlowRunId());
context.setUserId(flowContext.getUserId());
@ -367,15 +371,13 @@ class ApplicationEntityReader extends GenericEntityReader {
Connection conn, FilterList filterList) throws IOException {
Scan scan = new Scan();
TimelineReaderContext context = getContext();
if (context.getFlowRunId() != null) {
scan.setRowPrefixFilter(ApplicationRowKey.
getRowKeyPrefix(context.getClusterId(), context.getUserId(),
context.getFlowName(), context.getFlowRunId()));
} else {
scan.setRowPrefixFilter(ApplicationRowKey.
getRowKeyPrefix(context.getClusterId(), context.getUserId(),
context.getFlowName()));
}
// Whether or not flowRunID is null doesn't matter, the
// ApplicationRowKeyPrefix will do the right thing.
RowKeyPrefix<ApplicationRowKey> applicationRowKeyPrefix =
new ApplicationRowKeyPrefix(context.getClusterId(),
context.getUserId(), context.getFlowName(),
context.getFlowRunId());
scan.setRowPrefixFilter(applicationRowKeyPrefix.getRowKeyPrefix());
FilterList newList = new FilterList();
newList.addFilter(new PageFilter(getFilters().getLimit()));
if (filterList != null && !filterList.getFilters().isEmpty()) {
@ -409,15 +411,14 @@ class ApplicationEntityReader extends GenericEntityReader {
boolean checkIsRelatedTo =
!isSingleEntityRead() && filters.getIsRelatedTo() != null &&
filters.getIsRelatedTo().getFilterList().size() > 0;
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO) ||
checkIsRelatedTo) {
TimelineStorageUtils.readRelationship(
entity, result, ApplicationColumnPrefix.IS_RELATED_TO, true);
if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) {
readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
true);
if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity,
filters.getIsRelatedTo())) {
return null;
}
if (!TimelineStorageUtils.hasField(fieldsToRetrieve,
if (!hasField(fieldsToRetrieve,
Field.IS_RELATED_TO)) {
entity.getIsRelatedToEntities().clear();
}
@ -430,29 +431,27 @@ class ApplicationEntityReader extends GenericEntityReader {
boolean checkRelatesTo =
!isSingleEntityRead() && filters.getRelatesTo() != null &&
filters.getRelatesTo().getFilterList().size() > 0;
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO) ||
if (hasField(fieldsToRetrieve, Field.RELATES_TO) ||
checkRelatesTo) {
TimelineStorageUtils.readRelationship(
entity, result, ApplicationColumnPrefix.RELATES_TO, false);
readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
false);
if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity,
filters.getRelatesTo())) {
return null;
}
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
entity.getRelatesToEntities().clear();
}
}
// fetch info if fieldsToRetrieve contains INFO or ALL.
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
TimelineStorageUtils.readKeyValuePairs(
entity, result, ApplicationColumnPrefix.INFO, false);
if (hasField(fieldsToRetrieve, Field.INFO)) {
readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
}
// fetch configs if fieldsToRetrieve contains CONFIGS or ALL.
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.CONFIGS)) {
TimelineStorageUtils.readKeyValuePairs(
entity, result, ApplicationColumnPrefix.CONFIG, true);
if (hasField(fieldsToRetrieve, Field.CONFIGS)) {
readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
}
// fetch events and match event filters if they exist. If event filters do
@ -462,21 +461,19 @@ class ApplicationEntityReader extends GenericEntityReader {
boolean checkEvents =
!isSingleEntityRead() && filters.getEventFilters() != null &&
filters.getEventFilters().getFilterList().size() > 0;
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS) ||
checkEvents) {
TimelineStorageUtils.readEvents(
entity, result, ApplicationColumnPrefix.EVENT);
if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) {
readEvents(entity, result, ApplicationColumnPrefix.EVENT);
if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity,
filters.getEventFilters())) {
return null;
}
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
entity.getEvents().clear();
}
}
// fetch metrics if fieldsToRetrieve contains METRICS or ALL.
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.METRICS)) {
if (hasField(fieldsToRetrieve, Field.METRICS)) {
readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
}
return entity;

View File

@ -35,9 +35,11 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrie
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
import com.google.common.base.Preconditions;
@ -50,6 +52,12 @@ class FlowActivityEntityReader extends TimelineEntityReader {
private static final FlowActivityTable FLOW_ACTIVITY_TABLE =
new FlowActivityTable();
/**
* Used to convert Long key components to and from storage format.
*/
private final KeyConverter<Long> longKeyConverter = new LongKeyConverter();
public FlowActivityEntityReader(TimelineReaderContext ctxt,
TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
super(ctxt, entityFilters, toRetrieve, true);
@ -105,15 +113,14 @@ class FlowActivityEntityReader extends TimelineEntityReader {
if (getFilters().getCreatedTimeBegin() == 0L &&
getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) {
// All records have to be chosen.
scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));
scan.setRowPrefixFilter(new FlowActivityRowKeyPrefix(clusterId)
.getRowKeyPrefix());
} else {
scan.setStartRow(
FlowActivityRowKey.getRowKeyPrefix(clusterId,
getFilters().getCreatedTimeEnd()));
scan.setStopRow(
FlowActivityRowKey.getRowKeyPrefix(clusterId,
(getFilters().getCreatedTimeBegin() <= 0 ? 0 :
(getFilters().getCreatedTimeBegin() - 1))));
scan.setStartRow(new FlowActivityRowKeyPrefix(clusterId, getFilters()
.getCreatedTimeEnd()).getRowKeyPrefix());
scan.setStopRow(new FlowActivityRowKeyPrefix(clusterId, (getFilters()
.getCreatedTimeBegin() <= 0 ? 0
: (getFilters().getCreatedTimeBegin() - 1))).getRowKeyPrefix());
}
// use the page filter to limit the result to the page size
// the scanner may still return more than the limit; therefore we need to
@ -137,8 +144,7 @@ class FlowActivityEntityReader extends TimelineEntityReader {
// get the list of run ids along with the version that are associated with
// this flow on this day
Map<Long, Object> runIdsMap =
FlowActivityColumnPrefix.RUN_ID.readResults(result,
LongKeyConverter.getInstance());
FlowActivityColumnPrefix.RUN_ID.readResults(result, longKeyConverter);
for (Map.Entry<Long, Object> e : runIdsMap.entrySet()) {
Long runId = e.getKey();
String version = (String)e.getValue();

View File

@ -28,12 +28,12 @@ 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.BinaryPrefixComparator;
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.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
@ -43,11 +43,12 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilte
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.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
import org.apache.hadoop.yarn.webapp.BadRequestException;
@ -81,8 +82,8 @@ class FlowRunEntityReader extends TimelineEntityReader {
@Override
protected void validateParams() {
Preconditions.checkNotNull(getContext(), "context shouldn't be null");
Preconditions.checkNotNull(
getDataToRetrieve(), "data to retrieve 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().getUserId(),
@ -97,8 +98,8 @@ class FlowRunEntityReader extends TimelineEntityReader {
if (!isSingleEntityRead() && fieldsToRetrieve != null) {
for (Field field : fieldsToRetrieve) {
if (field != Field.ALL && field != Field.METRICS) {
throw new BadRequestException("Invalid field " + field +
" specified while querying flow runs.");
throw new BadRequestException("Invalid field " + field
+ " specified while querying flow runs.");
}
}
}
@ -119,23 +120,22 @@ class FlowRunEntityReader extends TimelineEntityReader {
Long createdTimeBegin = getFilters().getCreatedTimeBegin();
Long createdTimeEnd = getFilters().getCreatedTimeEnd();
if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
listBasedOnFilters.addFilter(
TimelineFilterUtils.createSingleColValueFiltersByRange(
FlowRunColumn.MIN_START_TIME, createdTimeBegin, createdTimeEnd));
listBasedOnFilters.addFilter(TimelineFilterUtils
.createSingleColValueFiltersByRange(FlowRunColumn.MIN_START_TIME,
createdTimeBegin, createdTimeEnd));
}
// Filter based on metric filters.
TimelineFilterList metricFilters = getFilters().getMetricFilters();
if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) {
listBasedOnFilters.addFilter(
TimelineFilterUtils.createHBaseFilterList(
FlowRunColumnPrefix.METRIC, metricFilters));
listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
FlowRunColumnPrefix.METRIC, metricFilters));
}
return listBasedOnFilters;
}
/**
* Add {@link QualifierFilter} filters to filter list for each column of
* flow run table.
* Add {@link QualifierFilter} filters to filter list for each column of flow
* run table.
*
* @return filter list to which qualifier filters have been added.
*/
@ -153,20 +153,19 @@ class FlowRunEntityReader extends TimelineEntityReader {
FilterList list = new FilterList(Operator.MUST_PASS_ONE);
// By default fetch everything in INFO column family.
FamilyFilter infoColumnFamily =
new FamilyFilter(CompareOp.EQUAL,
new BinaryComparator(FlowRunColumnFamily.INFO.getBytes()));
new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(
FlowRunColumnFamily.INFO.getBytes()));
TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
// If multiple entities have to be retrieved, check if metrics have to be
// retrieved and if not, add a filter so that metrics can be excluded.
// Metrics are always returned if we are reading a single entity.
if (!isSingleEntityRead() && !TimelineStorageUtils.hasField(
dataToRetrieve.getFieldsToRetrieve(), Field.METRICS)) {
if (!isSingleEntityRead()
&& !hasField(dataToRetrieve.getFieldsToRetrieve(), Field.METRICS)) {
FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE);
infoColFamilyList.addFilter(infoColumnFamily);
infoColFamilyList.addFilter(
new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator(
FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(""))));
infoColFamilyList.addFilter(new QualifierFilter(CompareOp.NOT_EQUAL,
new BinaryPrefixComparator(FlowRunColumnPrefix.METRIC
.getColumnPrefixBytes(""))));
list.addFilter(infoColFamilyList);
} else {
// Check if metricsToRetrieve are specified and if they are, create a
@ -176,14 +175,13 @@ class FlowRunEntityReader extends TimelineEntityReader {
// (in augmentParams()).
TimelineFilterList metricsToRetrieve =
dataToRetrieve.getMetricsToRetrieve();
if (metricsToRetrieve != null &&
!metricsToRetrieve.getFilterList().isEmpty()) {
if (metricsToRetrieve != null
&& !metricsToRetrieve.getFilterList().isEmpty()) {
FilterList infoColFamilyList = new FilterList();
infoColFamilyList.addFilter(infoColumnFamily);
FilterList columnsList = updateFixedColumns();
columnsList.addFilter(
TimelineFilterUtils.createHBaseFilterList(
FlowRunColumnPrefix.METRIC, metricsToRetrieve));
columnsList.addFilter(TimelineFilterUtils.createHBaseFilterList(
FlowRunColumnPrefix.METRIC, metricsToRetrieve));
infoColFamilyList.addFilter(columnsList);
list.addFilter(infoColFamilyList);
}
@ -195,9 +193,10 @@ class FlowRunEntityReader extends TimelineEntityReader {
protected Result getResult(Configuration hbaseConf, Connection conn,
FilterList filterList) throws IOException {
TimelineReaderContext context = getContext();
byte[] rowKey =
FlowRunRowKey.getRowKey(context.getClusterId(), context.getUserId(),
FlowRunRowKey flowRunRowKey =
new FlowRunRowKey(context.getClusterId(), context.getUserId(),
context.getFlowName(), context.getFlowRunId());
byte[] rowKey = flowRunRowKey.getRowKey();
Get get = new Get(rowKey);
get.setMaxVersions(Integer.MAX_VALUE);
if (filterList != null && !filterList.getFilters().isEmpty()) {
@ -207,13 +206,14 @@ class FlowRunEntityReader extends TimelineEntityReader {
}
@Override
protected ResultScanner getResults(Configuration hbaseConf,
Connection conn, FilterList filterList) throws IOException {
protected ResultScanner getResults(Configuration hbaseConf, Connection conn,
FilterList filterList) throws IOException {
Scan scan = new Scan();
TimelineReaderContext context = getContext();
scan.setRowPrefixFilter(
FlowRunRowKey.getRowKeyPrefix(context.getClusterId(),
context.getUserId(), context.getFlowName()));
RowKeyPrefix<FlowRunRowKey> flowRunRowKeyPrefix =
new FlowRunRowKeyPrefix(context.getClusterId(), context.getUserId(),
context.getFlowName());
scan.setRowPrefixFilter(flowRunRowKeyPrefix.getRowKeyPrefix());
FilterList newList = new FilterList();
newList.addFilter(new PageFilter(getFilters().getLimit()));
if (filterList != null && !filterList.getFilters().isEmpty()) {
@ -238,27 +238,27 @@ class FlowRunEntityReader extends TimelineEntityReader {
}
// read the start time
Long startTime = (Long)FlowRunColumn.MIN_START_TIME.readResult(result);
Long startTime = (Long) FlowRunColumn.MIN_START_TIME.readResult(result);
if (startTime != null) {
flowRun.setStartTime(startTime.longValue());
}
// read the end time if available
Long endTime = (Long)FlowRunColumn.MAX_END_TIME.readResult(result);
Long endTime = (Long) FlowRunColumn.MAX_END_TIME.readResult(result);
if (endTime != null) {
flowRun.setMaxEndTime(endTime.longValue());
}
// read the flow version
String version = (String)FlowRunColumn.FLOW_VERSION.readResult(result);
String version = (String) FlowRunColumn.FLOW_VERSION.readResult(result);
if (version != null) {
flowRun.setVersion(version);
}
// read metrics if its a single entity query or if METRICS are part of
// fieldsToRetrieve.
if (isSingleEntityRead() || TimelineStorageUtils.hasField(
getDataToRetrieve().getFieldsToRetrieve(), Field.METRICS)) {
if (isSingleEntityRead()
|| hasField(getDataToRetrieve().getFieldsToRetrieve(), Field.METRICS)) {
readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
import java.io.IOException;
import java.util.EnumSet;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
@ -28,11 +29,11 @@ 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.QualifierFilter;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
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;
@ -44,11 +45,16 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlow
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
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.EntityRowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
import org.apache.hadoop.yarn.webapp.NotFoundException;
@ -66,6 +72,12 @@ class GenericEntityReader extends TimelineEntityReader {
*/
private final AppToFlowTable appToFlowTable = new AppToFlowTable();
/**
* Used to convert strings key components to and from storage format.
*/
private final KeyConverter<String> stringKeyConverter =
new StringKeyConverter();
public GenericEntityReader(TimelineReaderContext ctxt,
TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve,
boolean sortedKeys) {
@ -95,32 +107,29 @@ class GenericEntityReader extends TimelineEntityReader {
long createdTimeBegin = filters.getCreatedTimeBegin();
long createdTimeEnd = filters.getCreatedTimeEnd();
if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
listBasedOnFilters.addFilter(
TimelineFilterUtils.createSingleColValueFiltersByRange(
EntityColumn.CREATED_TIME, createdTimeBegin, createdTimeEnd));
listBasedOnFilters.addFilter(TimelineFilterUtils
.createSingleColValueFiltersByRange(EntityColumn.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(
EntityColumnPrefix.METRIC, metricFilters));
listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
EntityColumnPrefix.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(
EntityColumnPrefix.CONFIG, configFilters));
listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
EntityColumnPrefix.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(
EntityColumnPrefix.INFO, infoFilters));
listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
EntityColumnPrefix.INFO, infoFilters));
}
return listBasedOnFilters;
}
@ -130,10 +139,10 @@ class GenericEntityReader extends TimelineEntityReader {
*
* @return true if we need to fetch some of the columns, false otherwise.
*/
private static boolean fetchPartialEventCols(TimelineFilterList eventFilters,
private boolean fetchPartialEventCols(TimelineFilterList eventFilters,
EnumSet<Field> fieldsToRetrieve) {
return (eventFilters != null && !eventFilters.getFilterList().isEmpty() &&
!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS));
!hasField(fieldsToRetrieve, Field.EVENTS));
}
/**
@ -141,10 +150,10 @@ class GenericEntityReader extends TimelineEntityReader {
*
* @return true if we need to fetch some of the columns, false otherwise.
*/
private static boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
private boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
EnumSet<Field> fieldsToRetrieve) {
return (relatesTo != null && !relatesTo.getFilterList().isEmpty() &&
!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO));
!hasField(fieldsToRetrieve, Field.RELATES_TO));
}
/**
@ -152,10 +161,10 @@ class GenericEntityReader extends TimelineEntityReader {
*
* @return true if we need to fetch some of the columns, false otherwise.
*/
private static boolean fetchPartialIsRelatedToCols(
TimelineFilterList isRelatedTo, EnumSet<Field> fieldsToRetrieve) {
private boolean fetchPartialIsRelatedToCols(TimelineFilterList isRelatedTo,
EnumSet<Field> fieldsToRetrieve) {
return (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty() &&
!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO));
!hasField(fieldsToRetrieve, Field.IS_RELATED_TO));
}
/**
@ -163,19 +172,20 @@ class GenericEntityReader extends TimelineEntityReader {
* relatesto and isrelatedto from info family.
*
* @return true, if we need to fetch only some of the columns, false if we
* need to fetch all the columns under info column family.
* need to fetch all the columns under info column family.
*/
protected boolean fetchPartialColsFromInfoFamily() {
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
TimelineEntityFilters filters = getFilters();
return fetchPartialEventCols(filters.getEventFilters(), fieldsToRetrieve) ||
fetchPartialRelatesToCols(filters.getRelatesTo(), fieldsToRetrieve) ||
fetchPartialIsRelatedToCols(filters.getIsRelatedTo(), fieldsToRetrieve);
return fetchPartialEventCols(filters.getEventFilters(), fieldsToRetrieve)
|| fetchPartialRelatesToCols(filters.getRelatesTo(), fieldsToRetrieve)
|| fetchPartialIsRelatedToCols(filters.getIsRelatedTo(),
fieldsToRetrieve);
}
/**
* Check if we need to create filter list based on fields. We need to create
* a filter list iff all fields need not be retrieved or we have some specific
* Check if we need to create filter list based on fields. We need to create a
* filter list iff all fields need not be retrieved or we have some specific
* fields or metrics to retrieve. We also need to create a filter list if we
* have relationships(relatesTo/isRelatedTo) and event filters specified for
* the query.
@ -188,22 +198,24 @@ class GenericEntityReader extends TimelineEntityReader {
// be retrieved, also check if we have some metrics or configs to
// retrieve specified for the query because then a filter list will have
// to be created.
boolean flag = !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) ||
(dataToRetrieve.getConfsToRetrieve() != null &&
!dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) ||
(dataToRetrieve.getMetricsToRetrieve() != null &&
!dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty());
boolean flag =
!dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL)
|| (dataToRetrieve.getConfsToRetrieve() != null && !dataToRetrieve
.getConfsToRetrieve().getFilterList().isEmpty())
|| (dataToRetrieve.getMetricsToRetrieve() != null && !dataToRetrieve
.getMetricsToRetrieve().getFilterList().isEmpty());
// Filters need to be checked only if we are reading multiple entities. If
// condition above is false, we check if there are relationships(relatesTo/
// isRelatedTo) and event filters specified for the query.
if (!flag && !isSingleEntityRead()) {
TimelineEntityFilters filters = getFilters();
flag = (filters.getEventFilters() != null &&
!filters.getEventFilters().getFilterList().isEmpty()) ||
(filters.getIsRelatedTo() != null &&
!filters.getIsRelatedTo().getFilterList().isEmpty()) ||
(filters.getRelatesTo() != null &&
!filters.getRelatesTo().getFilterList().isEmpty());
flag =
(filters.getEventFilters() != null && !filters.getEventFilters()
.getFilterList().isEmpty())
|| (filters.getIsRelatedTo() != null && !filters.getIsRelatedTo()
.getFilterList().isEmpty())
|| (filters.getRelatesTo() != null && !filters.getRelatesTo()
.getFilterList().isEmpty());
}
return flag;
}
@ -216,8 +228,8 @@ class GenericEntityReader extends TimelineEntityReader {
*/
protected void updateFixedColumns(FilterList list) {
for (EntityColumn column : EntityColumn.values()) {
list.addFilter(new QualifierFilter(CompareOp.EQUAL,
new BinaryComparator(column.getColumnQualifierBytes())));
list.addFilter(new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(
column.getColumnQualifierBytes())));
}
}
@ -226,30 +238,29 @@ class GenericEntityReader extends TimelineEntityReader {
* 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.
* application table, otherwise for entity table.
* @return filter list.
* @throws IOException if any problem occurs while creating filter list.
*/
private FilterList createFilterListForColsOfInfoFamily()
throws IOException {
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 (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
infoFamilyColsFilter.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
if (hasField(fieldsToRetrieve, Field.INFO)) {
infoFamilyColsFilter
.addFilter(TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.EQUAL, EntityColumnPrefix.INFO));
}
TimelineFilterList relatesTo = getFilters().getRelatesTo();
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
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, EntityColumnPrefix.RELATES_TO));
infoFamilyColsFilter.addFilter(TimelineFilterUtils
.createHBaseQualifierFilter(CompareOp.EQUAL,
EntityColumnPrefix.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
@ -257,17 +268,16 @@ class GenericEntityReader extends TimelineEntityReader {
// matched after fetching rows from HBase.
Set<String> relatesToCols =
TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo);
infoFamilyColsFilter.addFilter(
TimelineFilterUtils.createFiltersFromColumnQualifiers(
EntityColumnPrefix.RELATES_TO, relatesToCols));
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
EntityColumnPrefix.RELATES_TO, relatesToCols));
}
TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo();
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
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, EntityColumnPrefix.IS_RELATED_TO));
infoFamilyColsFilter.addFilter(TimelineFilterUtils
.createHBaseQualifierFilter(CompareOp.EQUAL,
EntityColumnPrefix.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
@ -275,27 +285,26 @@ class GenericEntityReader extends TimelineEntityReader {
// matched after fetching rows from HBase.
Set<String> isRelatedToCols =
TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo);
infoFamilyColsFilter.addFilter(
TimelineFilterUtils.createFiltersFromColumnQualifiers(
EntityColumnPrefix.IS_RELATED_TO, isRelatedToCols));
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
EntityColumnPrefix.IS_RELATED_TO, isRelatedToCols));
}
TimelineFilterList eventFilters = getFilters().getEventFilters();
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
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(
infoFamilyColsFilter
.addFilter(TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.EQUAL, EntityColumnPrefix.EVENT));
} else if (eventFilters != null && !eventFilters.getFilterList().isEmpty()){
} 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(
TimelineFilterUtils.createFiltersFromColumnQualifiers(
EntityColumnPrefix.EVENT, eventCols));
infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
EntityColumnPrefix.EVENT, eventCols));
}
return infoFamilyColsFilter;
}
@ -310,28 +319,28 @@ class GenericEntityReader extends TimelineEntityReader {
private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) {
EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
// Events not required.
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
infoColFamilyList.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.NOT_EQUAL, EntityColumnPrefix.EVENT));
if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
infoColFamilyList.addFilter(TimelineFilterUtils
.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
EntityColumnPrefix.EVENT));
}
// info not required.
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
infoColFamilyList.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.NOT_EQUAL, EntityColumnPrefix.INFO));
if (!hasField(fieldsToRetrieve, Field.INFO)) {
infoColFamilyList.addFilter(TimelineFilterUtils
.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
EntityColumnPrefix.INFO));
}
// is related to not required.
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
infoColFamilyList.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.NOT_EQUAL, EntityColumnPrefix.IS_RELATED_TO));
if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
infoColFamilyList.addFilter(TimelineFilterUtils
.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
EntityColumnPrefix.IS_RELATED_TO));
}
// relates to not required.
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
infoColFamilyList.addFilter(
TimelineFilterUtils.createHBaseQualifierFilter(
CompareOp.NOT_EQUAL, EntityColumnPrefix.RELATES_TO));
if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
infoColFamilyList.addFilter(TimelineFilterUtils
.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
EntityColumnPrefix.RELATES_TO));
}
}
@ -348,18 +357,18 @@ class GenericEntityReader extends TimelineEntityReader {
// 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(),
EntityColumnFamily.CONFIGS, EntityColumnPrefix.CONFIG));
listBasedOnFields.addFilter(TimelineFilterUtils
.createFilterForConfsOrMetricsToRetrieve(
dataToRetrieve.getConfsToRetrieve(), EntityColumnFamily.CONFIGS,
EntityColumnPrefix.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(
listBasedOnFields.addFilter(TimelineFilterUtils
.createFilterForConfsOrMetricsToRetrieve(
dataToRetrieve.getMetricsToRetrieve(),
EntityColumnFamily.METRICS, EntityColumnPrefix.METRIC));
}
@ -375,8 +384,8 @@ class GenericEntityReader extends TimelineEntityReader {
FilterList infoColFamilyList = new FilterList();
// By default fetch everything in INFO column family.
FamilyFilter infoColumnFamily =
new FamilyFilter(CompareOp.EQUAL,
new BinaryComparator(EntityColumnFamily.INFO.getBytes()));
new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(
EntityColumnFamily.INFO.getBytes()));
infoColFamilyList.addFilter(infoColumnFamily);
if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) {
// We can fetch only some of the columns from info family.
@ -394,27 +403,27 @@ class GenericEntityReader extends TimelineEntityReader {
/**
* Looks up flow context from AppToFlow table.
*
* @param clusterId Cluster Id.
* @param appId App Id.
* @param appToFlowRowKey to identify Cluster and App Ids.
* @param hbaseConf HBase configuration.
* @param conn HBase Connection.
* @return flow context information.
* @throws IOException if any problem occurs while fetching flow information.
*/
protected FlowContext lookupFlowContext(String clusterId, String appId,
protected FlowContext lookupFlowContext(AppToFlowRowKey appToFlowRowKey,
Configuration hbaseConf, Connection conn) throws IOException {
byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
byte[] rowKey = appToFlowRowKey.getRowKey();
Get get = new Get(rowKey);
Result result = appToFlowTable.getResult(hbaseConf, conn, get);
if (result != null && !result.isEmpty()) {
return new FlowContext(
AppToFlowColumn.USER_ID.readResult(result).toString(),
AppToFlowColumn.FLOW_ID.readResult(result).toString(),
((Number)AppToFlowColumn.FLOW_RUN_ID.readResult(result)).longValue());
return new FlowContext(AppToFlowColumn.USER_ID.readResult(result)
.toString(), AppToFlowColumn.FLOW_ID.readResult(result).toString(),
((Number) AppToFlowColumn.FLOW_RUN_ID.readResult(result))
.longValue());
} else {
throw new NotFoundException(
"Unable to find the context flow ID and flow run ID for clusterId=" +
clusterId + ", appId=" + appId);
"Unable to find the context flow ID and flow run ID for clusterId="
+ appToFlowRowKey.getClusterId() + ", appId="
+ appToFlowRowKey.getAppId());
}
}
@ -425,17 +434,21 @@ class GenericEntityReader extends TimelineEntityReader {
private final String userId;
private final String flowName;
private final Long flowRunId;
public FlowContext(String user, String flowName, Long flowRunId) {
this.userId = user;
this.flowName = flowName;
this.flowRunId = flowRunId;
}
protected String getUserId() {
return userId;
}
protected String getFlowName() {
return flowName;
}
protected Long getFlowRunId() {
return flowRunId;
}
@ -444,8 +457,8 @@ class GenericEntityReader extends TimelineEntityReader {
@Override
protected void validateParams() {
Preconditions.checkNotNull(getContext(), "context shouldn't be null");
Preconditions.checkNotNull(
getDataToRetrieve(), "data to retrieve 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().getAppId(),
@ -463,11 +476,13 @@ class GenericEntityReader extends TimelineEntityReader {
throws IOException {
TimelineReaderContext context = getContext();
// In reality all three should be null or neither should be null
if (context.getFlowName() == null || context.getFlowRunId() == null ||
context.getUserId() == null) {
if (context.getFlowName() == null || context.getFlowRunId() == null
|| context.getUserId() == null) {
// Get flow context information from AppToFlow table.
FlowContext flowContext = lookupFlowContext(
context.getClusterId(), context.getAppId(), hbaseConf, conn);
AppToFlowRowKey appToFlowRowKey =
new AppToFlowRowKey(context.getClusterId(), context.getAppId());
FlowContext flowContext =
lookupFlowContext(appToFlowRowKey, hbaseConf, conn);
context.setFlowName(flowContext.flowName);
context.setFlowRunId(flowContext.flowRunId);
context.setUserId(flowContext.userId);
@ -485,9 +500,9 @@ class GenericEntityReader extends TimelineEntityReader {
FilterList filterList) throws IOException {
TimelineReaderContext context = getContext();
byte[] rowKey =
EntityRowKey.getRowKey(context.getClusterId(), context.getUserId(),
new EntityRowKey(context.getClusterId(), context.getUserId(),
context.getFlowName(), context.getFlowRunId(), context.getAppId(),
context.getEntityType(), context.getEntityId());
context.getEntityType(), context.getEntityId()).getRowKey();
Get get = new Get(rowKey);
get.setMaxVersions(getDataToRetrieve().getMetricsLimit());
if (filterList != null && !filterList.getFilters().isEmpty()) {
@ -497,15 +512,17 @@ class GenericEntityReader extends TimelineEntityReader {
}
@Override
protected ResultScanner getResults(Configuration hbaseConf,
Connection conn, FilterList filterList) throws IOException {
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();
scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
context.getClusterId(), context.getUserId(), context.getFlowName(),
context.getFlowRunId(), context.getAppId(), context.getEntityType()));
RowKeyPrefix<EntityRowKey> entityRowKeyPrefix =
new EntityRowKeyPrefix(context.getClusterId(), context.getUserId(),
context.getFlowName(), context.getFlowRunId(), context.getAppId(),
context.getEntityType());
scan.setRowPrefixFilter(entityRowKeyPrefix.getRowKeyPrefix());
scan.setMaxVersions(getDataToRetrieve().getMetricsLimit());
if (filterList != null && !filterList.getFilters().isEmpty()) {
scan.setFilter(filterList);
@ -535,18 +552,16 @@ class GenericEntityReader extends TimelineEntityReader {
// locally as relevant HBase filters to filter out rows on the basis of
// isRelatedTo are not set in HBase scan.
boolean checkIsRelatedTo =
!isSingleEntityRead() && filters.getIsRelatedTo() != null &&
filters.getIsRelatedTo().getFilterList().size() > 0;
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO) ||
checkIsRelatedTo) {
TimelineStorageUtils.readRelationship(
entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity,
filters.getIsRelatedTo())) {
!isSingleEntityRead() && filters.getIsRelatedTo() != null
&& filters.getIsRelatedTo().getFilterList().size() > 0;
if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) {
readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
if (checkIsRelatedTo
&& !TimelineStorageUtils.matchIsRelatedTo(entity,
filters.getIsRelatedTo())) {
return null;
}
if (!TimelineStorageUtils.hasField(fieldsToRetrieve,
Field.IS_RELATED_TO)) {
if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
entity.getIsRelatedToEntities().clear();
}
}
@ -556,31 +571,29 @@ class GenericEntityReader extends TimelineEntityReader {
// 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 (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO) ||
checkRelatesTo) {
TimelineStorageUtils.readRelationship(
entity, result, EntityColumnPrefix.RELATES_TO, false);
if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity,
filters.getRelatesTo())) {
!isSingleEntityRead() && filters.getRelatesTo() != null
&& filters.getRelatesTo().getFilterList().size() > 0;
if (hasField(fieldsToRetrieve, Field.RELATES_TO)
|| checkRelatesTo) {
readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
if (checkRelatesTo
&& !TimelineStorageUtils.matchRelatesTo(entity,
filters.getRelatesTo())) {
return null;
}
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
entity.getRelatesToEntities().clear();
}
}
// fetch info if fieldsToRetrieve contains INFO or ALL.
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
TimelineStorageUtils.readKeyValuePairs(
entity, result, EntityColumnPrefix.INFO, false);
if (hasField(fieldsToRetrieve, Field.INFO)) {
readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
}
// fetch configs if fieldsToRetrieve contains CONFIGS or ALL.
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.CONFIGS)) {
TimelineStorageUtils.readKeyValuePairs(
entity, result, EntityColumnPrefix.CONFIG, true);
if (hasField(fieldsToRetrieve, Field.CONFIGS)) {
readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
}
// fetch events and match event filters if they exist. If event filters do
@ -588,24 +601,48 @@ class GenericEntityReader extends TimelineEntityReader {
// 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 (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS) ||
checkEvents) {
TimelineStorageUtils.readEvents(entity, result, EntityColumnPrefix.EVENT);
if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity,
filters.getEventFilters())) {
!isSingleEntityRead() && filters.getEventFilters() != null
&& filters.getEventFilters().getFilterList().size() > 0;
if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) {
readEvents(entity, result, EntityColumnPrefix.EVENT);
if (checkEvents
&& !TimelineStorageUtils.matchEventFilters(entity,
filters.getEventFilters())) {
return null;
}
if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
entity.getEvents().clear();
}
}
// fetch metrics if fieldsToRetrieve contains METRICS or ALL.
if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.METRICS)) {
if (hasField(fieldsToRetrieve, Field.METRICS)) {
readMetrics(entity, result, EntityColumnPrefix.METRIC);
}
return entity;
}
/**
* Helper method for reading key-value pairs for either info or config.
*
* @param <T> Describes the type of column prefix.
* @param entity entity to fill.
* @param result result from HBase.
* @param prefix column prefix.
* @param isConfig if true, means we are reading configs, otherwise info.
* @throws IOException if any problem is encountered while reading result.
*/
protected <T> void readKeyValuePairs(TimelineEntity entity, Result result,
ColumnPrefix<T> prefix, boolean isConfig) throws IOException {
// info and configuration are of type Map<String, Object or String>
Map<String, Object> columns =
prefix.readResults(result, stringKeyConverter);
if (isConfig) {
for (Map.Entry<String, Object> column : columns.entrySet()) {
entity.addConfig(column.getKey(), column.getValue().toString());
}
} else {
entity.addInfo(columns);
}
}
}

View File

@ -18,6 +18,9 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
import java.io.IOException;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
@ -30,15 +33,27 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.hbase.filter.QualifierFilter;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
/**
* The base class for reading and deserializing timeline entities from the
@ -67,6 +82,12 @@ public abstract class TimelineEntityReader {
*/
private boolean sortedKeys = false;
/**
* Used to convert strings key components to and from storage format.
*/
private final KeyConverter<String> stringKeyConverter =
new StringKeyConverter();
/**
* Instantiates a reader for multiple-entity reads.
*
@ -331,7 +352,7 @@ public abstract class TimelineEntityReader {
ColumnPrefix<?> columnPrefix) throws IOException {
NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
columnPrefix.readResultsWithTimestamps(
result, StringKeyConverter.getInstance());
result, stringKeyConverter);
for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
metricsResult.entrySet()) {
TimelineMetric metric = new TimelineMetric();
@ -359,4 +380,117 @@ public abstract class TimelineEntityReader {
protected void setTable(BaseTable<?> baseTable) {
this.table = baseTable;
}
/**
* Check if we have a certain field amongst fields to retrieve. This method
* checks against {@link Field#ALL} as well because that would mean field
* passed needs to be matched.
*
* @param fieldsToRetrieve fields to be retrieved.
* @param requiredField fields to be checked in fieldsToRetrieve.
* @return true if has the required field, false otherwise.
*/
protected boolean hasField(EnumSet<Field> fieldsToRetrieve,
Field requiredField) {
return fieldsToRetrieve.contains(Field.ALL) ||
fieldsToRetrieve.contains(requiredField);
}
/**
* Create a filter list of qualifier filters based on passed set of columns.
*
* @param <T> Describes the type of column prefix.
* @param colPrefix Column Prefix.
* @param columns set of column qualifiers.
* @return filter list.
*/
protected <T> FilterList createFiltersFromColumnQualifiers(
ColumnPrefix<T> colPrefix, Set<String> columns) {
FilterList list = new FilterList(Operator.MUST_PASS_ONE);
for (String column : columns) {
// For columns which have compound column qualifiers (eg. events), we need
// to include the required separator.
byte[] compoundColQual = createColQualifierPrefix(colPrefix, column);
list.addFilter(new QualifierFilter(CompareOp.EQUAL,
new BinaryPrefixComparator(colPrefix
.getColumnPrefixBytes(compoundColQual))));
}
return list;
}
protected <T> byte[] createColQualifierPrefix(ColumnPrefix<T> colPrefix,
String column) {
if (colPrefix == ApplicationColumnPrefix.EVENT
|| colPrefix == EntityColumnPrefix.EVENT) {
return new EventColumnName(column, null, null).getColumnQualifier();
} else {
return stringKeyConverter.encode(column);
}
}
/**
* Helper method for reading relationship.
*
* @param <T> Describes the type of column prefix.
* @param entity entity to fill.
* @param result result from HBase.
* @param prefix column prefix.
* @param isRelatedTo if true, means relationship is to be added to
* isRelatedTo, otherwise its added to relatesTo.
* @throws IOException if any problem is encountered while reading result.
*/
protected <T> void readRelationship(TimelineEntity entity, Result result,
ColumnPrefix<T> prefix, boolean isRelatedTo) throws IOException {
// isRelatedTo and relatesTo are of type Map<String, Set<String>>
Map<String, Object> columns =
prefix.readResults(result, stringKeyConverter);
for (Map.Entry<String, Object> column : columns.entrySet()) {
for (String id : Separator.VALUES.splitEncoded(column.getValue()
.toString())) {
if (isRelatedTo) {
entity.addIsRelatedToEntity(column.getKey(), id);
} else {
entity.addRelatesToEntity(column.getKey(), id);
}
}
}
}
/**
* Read events from the entity table or the application table. The column name
* is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted
* if there is no info associated with the event.
*
* @param <T> Describes the type of column prefix.
* @param entity entity to fill.
* @param result HBase Result.
* @param prefix column prefix.
* @throws IOException if any problem is encountered while reading result.
*/
protected static <T> void readEvents(TimelineEntity entity, Result result,
ColumnPrefix<T> prefix) throws IOException {
Map<String, TimelineEvent> eventsMap = new HashMap<>();
Map<EventColumnName, Object> eventsResult =
prefix.readResults(result, new EventColumnNameConverter());
for (Map.Entry<EventColumnName, Object>
eventResult : eventsResult.entrySet()) {
EventColumnName eventColumnName = eventResult.getKey();
String key = eventColumnName.getId() +
Long.toString(eventColumnName.getTimestamp());
// Retrieve previously seen event to add to it
TimelineEvent event = eventsMap.get(key);
if (event == null) {
// First time we're seeing this event, add it to the eventsMap
event = new TimelineEvent();
event.setId(eventColumnName.getId());
event.setTimestamp(eventColumnName.getTimestamp());
eventsMap.put(key, event);
}
if (eventColumnName.getInfoKey() != null) {
event.addInfo(eventColumnName.getInfoKey(), eventResult.getValue());
}
}
Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
entity.addEvents(eventsSet);
}
}

View File

@ -24,220 +24,13 @@ import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKeyConverter;
import org.junit.Test;
public class TestKeyConverters {
private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue();
private final static byte[] QUALIFIER_SEP_BYTES =
Bytes.toBytes(QUALIFIER_SEP);
private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster";
private final static String USER = QUALIFIER_SEP + "user";
private final static String FLOW_NAME =
"dummy_" + QUALIFIER_SEP + "flow" + QUALIFIER_SEP;
private final static Long FLOW_RUN_ID;
private final static String APPLICATION_ID;
static {
long runid = Long.MAX_VALUE - 900L;
byte[] longMaxByteArr = Bytes.toBytes(Long.MAX_VALUE);
byte[] byteArr = Bytes.toBytes(runid);
int sepByteLen = QUALIFIER_SEP_BYTES.length;
if (sepByteLen <= byteArr.length) {
for (int i = 0; i < sepByteLen; i++) {
byteArr[i] = (byte)(longMaxByteArr[i] - QUALIFIER_SEP_BYTES[i]);
}
}
FLOW_RUN_ID = Bytes.toLong(byteArr);
long clusterTs = System.currentTimeMillis();
byteArr = Bytes.toBytes(clusterTs);
if (sepByteLen <= byteArr.length) {
for (int i = 0; i < sepByteLen; i++) {
byteArr[byteArr.length - sepByteLen + i] =
(byte)(longMaxByteArr[byteArr.length - sepByteLen + i] -
QUALIFIER_SEP_BYTES[i]);
}
}
clusterTs = Bytes.toLong(byteArr);
int seqId = 222;
APPLICATION_ID = ApplicationId.newInstance(clusterTs, seqId).toString();
}
private static void verifyRowPrefixBytes(byte[] byteRowKeyPrefix) {
int sepLen = QUALIFIER_SEP_BYTES.length;
for (int i = 0; i < sepLen; i++) {
assertTrue("Row key prefix not encoded properly.",
byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen + i] ==
QUALIFIER_SEP_BYTES[i]);
}
}
@Test
public void testFlowActivityRowKeyConverter() {
Long ts = TimelineStorageUtils.getTopOfTheDayTimestamp(1459900830000L);
byte[] byteRowKey = FlowActivityRowKeyConverter.getInstance().encode(
new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME));
FlowActivityRowKey rowKey =
FlowActivityRowKeyConverter.getInstance().decode(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(ts, rowKey.getDayTimestamp());
assertEquals(USER, rowKey.getUserId());
assertEquals(FLOW_NAME, rowKey.getFlowName());
byte[] byteRowKeyPrefix = FlowActivityRowKeyConverter.getInstance().encode(
new FlowActivityRowKey(CLUSTER, null, null, null));
byte[][] splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
assertEquals(2, splits.length);
assertEquals(0, splits[1].length);
assertEquals(CLUSTER,
Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
verifyRowPrefixBytes(byteRowKeyPrefix);
byteRowKeyPrefix = FlowActivityRowKeyConverter.getInstance().encode(
new FlowActivityRowKey(CLUSTER, ts, null, null));
splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE });
assertEquals(3, splits.length);
assertEquals(0, splits[2].length);
assertEquals(CLUSTER,
Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
assertEquals(ts, (Long) TimelineStorageUtils.invertLong(
Bytes.toLong(splits[1])));
verifyRowPrefixBytes(byteRowKeyPrefix);
}
@Test
public void testFlowRunRowKeyConverter() {
byte[] byteRowKey = FlowRunRowKeyConverter.getInstance().encode(
new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID));
FlowRunRowKey rowKey =
FlowRunRowKeyConverter.getInstance().decode(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(USER, rowKey.getUserId());
assertEquals(FLOW_NAME, rowKey.getFlowName());
assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
byte[] byteRowKeyPrefix = FlowRunRowKeyConverter.getInstance().encode(
new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, null));
byte[][] splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
assertEquals(4, splits.length);
assertEquals(0, splits[3].length);
assertEquals(FLOW_NAME,
Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
verifyRowPrefixBytes(byteRowKeyPrefix);
}
@Test
public void testApplicationRowKeyConverter() {
byte[] byteRowKey = ApplicationRowKeyConverter.getInstance().encode(
new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
APPLICATION_ID));
ApplicationRowKey rowKey =
ApplicationRowKeyConverter.getInstance().decode(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(USER, rowKey.getUserId());
assertEquals(FLOW_NAME, rowKey.getFlowName());
assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
assertEquals(APPLICATION_ID, rowKey.getAppId());
byte[] byteRowKeyPrefix = ApplicationRowKeyConverter.getInstance().encode(
new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, null));
byte[][] splits =
Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
Separator.VARIABLE_SIZE });
assertEquals(5, splits.length);
assertEquals(0, splits[4].length);
assertEquals(FLOW_NAME,
Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
assertEquals(FLOW_RUN_ID, (Long)TimelineStorageUtils.invertLong(
Bytes.toLong(splits[3])));
verifyRowPrefixBytes(byteRowKeyPrefix);
byteRowKeyPrefix = ApplicationRowKeyConverter.getInstance().encode(
new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, null, null));
splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
assertEquals(4, splits.length);
assertEquals(0, splits[3].length);
assertEquals(FLOW_NAME,
Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
verifyRowPrefixBytes(byteRowKeyPrefix);
}
@Test
public void testEntityRowKeyConverter() {
String entityId = "!ent!ity!!id!";
String entityType = "entity!Type";
byte[] byteRowKey = EntityRowKeyConverter.getInstance().encode(
new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
entityType, entityId));
EntityRowKey rowKey =
EntityRowKeyConverter.getInstance().decode(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(USER, rowKey.getUserId());
assertEquals(FLOW_NAME, rowKey.getFlowName());
assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
assertEquals(APPLICATION_ID, rowKey.getAppId());
assertEquals(entityType, rowKey.getEntityType());
assertEquals(entityId, rowKey.getEntityId());
byte[] byteRowKeyPrefix = EntityRowKeyConverter.getInstance().encode(
new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
entityType, null));
byte[][] splits =
Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE });
assertEquals(7, splits.length);
assertEquals(0, splits[6].length);
assertEquals(APPLICATION_ID,
AppIdKeyConverter.getInstance().decode(splits[4]));
assertEquals(entityType, Separator.QUALIFIERS.decode(
Bytes.toString(splits[5])));
verifyRowPrefixBytes(byteRowKeyPrefix);
byteRowKeyPrefix = EntityRowKeyConverter.getInstance().encode(
new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
null, null));
splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE });
assertEquals(6, splits.length);
assertEquals(0, splits[5].length);
assertEquals(APPLICATION_ID,
AppIdKeyConverter.getInstance().decode(splits[4]));
verifyRowPrefixBytes(byteRowKeyPrefix);
}
@Test
public void testAppToFlowRowKeyConverter() {
byte[] byteRowKey = AppToFlowRowKeyConverter.getInstance().encode(
new AppToFlowRowKey(CLUSTER, APPLICATION_ID));
AppToFlowRowKey rowKey =
AppToFlowRowKeyConverter.getInstance().decode(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(APPLICATION_ID, rowKey.getAppId());
}
@Test
public void testAppIdKeyConverter() {
AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter();
long currentTs = System.currentTimeMillis();
ApplicationId appId1 = ApplicationId.newInstance(currentTs, 1);
ApplicationId appId2 = ApplicationId.newInstance(currentTs, 2);
@ -245,18 +38,19 @@ public class TestKeyConverters {
String appIdStr1 = appId1.toString();
String appIdStr2 = appId2.toString();
String appIdStr3 = appId3.toString();
byte[] appIdBytes1 = AppIdKeyConverter.getInstance().encode(appIdStr1);
byte[] appIdBytes2 = AppIdKeyConverter.getInstance().encode(appIdStr2);
byte[] appIdBytes3 = AppIdKeyConverter.getInstance().encode(appIdStr3);
byte[] appIdBytes1 = appIdKeyConverter.encode(appIdStr1);
byte[] appIdBytes2 = appIdKeyConverter.encode(appIdStr2);
byte[] appIdBytes3 = appIdKeyConverter.encode(appIdStr3);
// App ids' should be encoded in a manner wherein descending order
// is maintained.
assertTrue("Ordering of app ids' is incorrect",
Bytes.compareTo(appIdBytes1, appIdBytes2) > 0 &&
Bytes.compareTo(appIdBytes1, appIdBytes3) > 0 &&
Bytes.compareTo(appIdBytes2, appIdBytes3) > 0);
String decodedAppId1 = AppIdKeyConverter.getInstance().decode(appIdBytes1);
String decodedAppId2 = AppIdKeyConverter.getInstance().decode(appIdBytes2);
String decodedAppId3 = AppIdKeyConverter.getInstance().decode(appIdBytes3);
assertTrue(
"Ordering of app ids' is incorrect",
Bytes.compareTo(appIdBytes1, appIdBytes2) > 0
&& Bytes.compareTo(appIdBytes1, appIdBytes3) > 0
&& Bytes.compareTo(appIdBytes2, appIdBytes3) > 0);
String decodedAppId1 = appIdKeyConverter.decode(appIdBytes1);
String decodedAppId2 = appIdKeyConverter.decode(appIdBytes2);
String decodedAppId3 = appIdKeyConverter.decode(appIdBytes3);
assertTrue("Decoded app id is not same as the app id encoded",
appIdStr1.equals(decodedAppId1));
assertTrue("Decoded app id is not same as the app id encoded",
@ -273,21 +67,64 @@ public class TestKeyConverters {
Bytes.createMaxByteArray(Bytes.SIZEOF_LONG - valSepBytes.length);
byte[] ts = Bytes.add(valSepBytes, maxByteArr);
Long eventTs = Bytes.toLong(ts);
byte[] byteEventColName = EventColumnNameConverter.getInstance().encode(
new EventColumnName(eventId, eventTs, null));
byte[] byteEventColName =
new EventColumnName(eventId, eventTs, null).getColumnQualifier();
KeyConverter<EventColumnName> eventColumnNameConverter =
new EventColumnNameConverter();
EventColumnName eventColName =
EventColumnNameConverter.getInstance().decode(byteEventColName);
eventColumnNameConverter.decode(byteEventColName);
assertEquals(eventId, eventColName.getId());
assertEquals(eventTs, eventColName.getTimestamp());
assertNull(eventColName.getInfoKey());
String infoKey = "f=oo_event_in=fo=_key";
byteEventColName = EventColumnNameConverter.getInstance().encode(
new EventColumnName(eventId, eventTs, infoKey));
eventColName =
EventColumnNameConverter.getInstance().decode(byteEventColName);
byteEventColName =
new EventColumnName(eventId, eventTs, infoKey).getColumnQualifier();
eventColName = eventColumnNameConverter.decode(byteEventColName);
assertEquals(eventId, eventColName.getId());
assertEquals(eventTs, eventColName.getTimestamp());
assertEquals(infoKey, eventColName.getInfoKey());
}
@Test
public void testLongKeyConverter() {
LongKeyConverter longKeyConverter = new LongKeyConverter();
confirmLongKeyConverter(longKeyConverter, Long.MIN_VALUE);
confirmLongKeyConverter(longKeyConverter, -1234567890L);
confirmLongKeyConverter(longKeyConverter, -128L);
confirmLongKeyConverter(longKeyConverter, -127L);
confirmLongKeyConverter(longKeyConverter, -1L);
confirmLongKeyConverter(longKeyConverter, 0L);
confirmLongKeyConverter(longKeyConverter, 1L);
confirmLongKeyConverter(longKeyConverter, 127L);
confirmLongKeyConverter(longKeyConverter, 128L);
confirmLongKeyConverter(longKeyConverter, 1234567890L);
confirmLongKeyConverter(longKeyConverter, Long.MAX_VALUE);
}
private void confirmLongKeyConverter(LongKeyConverter longKeyConverter,
Long testValue) {
Long decoded = longKeyConverter.decode(longKeyConverter.encode(testValue));
assertEquals(testValue, decoded);
}
@Test
public void testStringKeyConverter() {
StringKeyConverter stringKeyConverter = new StringKeyConverter();
String phrase = "QuackAttack now!";
for (int i = 0; i < phrase.length(); i++) {
String sub = phrase.substring(i, phrase.length());
confirmStrignKeyConverter(stringKeyConverter, sub);
confirmStrignKeyConverter(stringKeyConverter, sub + sub);
}
}
private void confirmStrignKeyConverter(StringKeyConverter stringKeyConverter,
String testValue) {
String decoded =
stringKeyConverter.decode(stringKeyConverter.encode(testValue));
assertEquals(testValue, decoded);
}
}

View File

@ -0,0 +1,244 @@
/**
* 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.common;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
import org.junit.Test;
public class TestRowKeys {
private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue();
private final static byte[] QUALIFIER_SEP_BYTES = Bytes
.toBytes(QUALIFIER_SEP);
private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster";
private final static String USER = QUALIFIER_SEP + "user";
private final static String FLOW_NAME = "dummy_" + QUALIFIER_SEP + "flow"
+ QUALIFIER_SEP;
private final static Long FLOW_RUN_ID;
private final static String APPLICATION_ID;
static {
long runid = Long.MAX_VALUE - 900L;
byte[] longMaxByteArr = Bytes.toBytes(Long.MAX_VALUE);
byte[] byteArr = Bytes.toBytes(runid);
int sepByteLen = QUALIFIER_SEP_BYTES.length;
if (sepByteLen <= byteArr.length) {
for (int i = 0; i < sepByteLen; i++) {
byteArr[i] = (byte) (longMaxByteArr[i] - QUALIFIER_SEP_BYTES[i]);
}
}
FLOW_RUN_ID = Bytes.toLong(byteArr);
long clusterTs = System.currentTimeMillis();
byteArr = Bytes.toBytes(clusterTs);
if (sepByteLen <= byteArr.length) {
for (int i = 0; i < sepByteLen; i++) {
byteArr[byteArr.length - sepByteLen + i] =
(byte) (longMaxByteArr[byteArr.length - sepByteLen + i] - QUALIFIER_SEP_BYTES[i]);
}
}
clusterTs = Bytes.toLong(byteArr);
int seqId = 222;
APPLICATION_ID = ApplicationId.newInstance(clusterTs, seqId).toString();
}
private static void verifyRowPrefixBytes(byte[] byteRowKeyPrefix) {
int sepLen = QUALIFIER_SEP_BYTES.length;
for (int i = 0; i < sepLen; i++) {
assertTrue(
"Row key prefix not encoded properly.",
byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen + i] == QUALIFIER_SEP_BYTES[i]);
}
}
@Test
public void testApplicationRowKey() {
byte[] byteRowKey =
new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
APPLICATION_ID).getRowKey();
ApplicationRowKey rowKey = ApplicationRowKey.parseRowKey(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(USER, rowKey.getUserId());
assertEquals(FLOW_NAME, rowKey.getFlowName());
assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
assertEquals(APPLICATION_ID, rowKey.getAppId());
byte[] byteRowKeyPrefix =
new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID)
.getRowKeyPrefix();
byte[][] splits =
Separator.QUALIFIERS.split(byteRowKeyPrefix,
new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
Separator.VARIABLE_SIZE });
assertEquals(5, splits.length);
assertEquals(0, splits[4].length);
assertEquals(FLOW_NAME,
Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
assertEquals(FLOW_RUN_ID,
(Long) LongConverter.invertLong(Bytes.toLong(splits[3])));
verifyRowPrefixBytes(byteRowKeyPrefix);
byteRowKeyPrefix =
new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME).getRowKeyPrefix();
splits =
Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
assertEquals(4, splits.length);
assertEquals(0, splits[3].length);
assertEquals(FLOW_NAME,
Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
verifyRowPrefixBytes(byteRowKeyPrefix);
}
/**
* Tests the converters indirectly through the public methods of the
* corresponding rowkey.
*/
@Test
public void testAppToFlowRowKey() {
byte[] byteRowKey = new AppToFlowRowKey(CLUSTER,
APPLICATION_ID).getRowKey();
AppToFlowRowKey rowKey = AppToFlowRowKey.parseRowKey(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(APPLICATION_ID, rowKey.getAppId());
}
@Test
public void testEntityRowKey() {
String entityId = "!ent!ity!!id!";
String entityType = "entity!Type";
byte[] byteRowKey =
new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
entityType, entityId).getRowKey();
EntityRowKey rowKey = EntityRowKey.parseRowKey(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(USER, rowKey.getUserId());
assertEquals(FLOW_NAME, rowKey.getFlowName());
assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
assertEquals(APPLICATION_ID, rowKey.getAppId());
assertEquals(entityType, rowKey.getEntityType());
assertEquals(entityId, rowKey.getEntityId());
byte[] byteRowKeyPrefix =
new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
APPLICATION_ID, entityType).getRowKeyPrefix();
byte[][] splits =
Separator.QUALIFIERS.split(
byteRowKeyPrefix,
new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE });
assertEquals(7, splits.length);
assertEquals(0, splits[6].length);
assertEquals(APPLICATION_ID, new AppIdKeyConverter().decode(splits[4]));
assertEquals(entityType,
Separator.QUALIFIERS.decode(Bytes.toString(splits[5])));
verifyRowPrefixBytes(byteRowKeyPrefix);
byteRowKeyPrefix =
new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
APPLICATION_ID).getRowKeyPrefix();
splits =
Separator.QUALIFIERS.split(
byteRowKeyPrefix,
new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE });
assertEquals(6, splits.length);
assertEquals(0, splits[5].length);
AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter();
assertEquals(APPLICATION_ID, appIdKeyConverter.decode(splits[4]));
verifyRowPrefixBytes(byteRowKeyPrefix);
}
@Test
public void testFlowActivityRowKey() {
Long ts = 1459900830000L;
Long dayTimestamp = TimelineStorageUtils.getTopOfTheDayTimestamp(ts);
byte[] byteRowKey =
new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME).getRowKey();
FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(dayTimestamp, rowKey.getDayTimestamp());
assertEquals(USER, rowKey.getUserId());
assertEquals(FLOW_NAME, rowKey.getFlowName());
byte[] byteRowKeyPrefix =
new FlowActivityRowKeyPrefix(CLUSTER).getRowKeyPrefix();
byte[][] splits =
Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
assertEquals(2, splits.length);
assertEquals(0, splits[1].length);
assertEquals(CLUSTER,
Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
verifyRowPrefixBytes(byteRowKeyPrefix);
byteRowKeyPrefix =
new FlowActivityRowKeyPrefix(CLUSTER, ts).getRowKeyPrefix();
splits =
Separator.QUALIFIERS.split(byteRowKeyPrefix,
new int[] { Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
Separator.VARIABLE_SIZE });
assertEquals(3, splits.length);
assertEquals(0, splits[2].length);
assertEquals(CLUSTER,
Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
assertEquals(ts,
(Long) LongConverter.invertLong(Bytes.toLong(splits[1])));
verifyRowPrefixBytes(byteRowKeyPrefix);
}
@Test
public void testFlowRunRowKey() {
byte[] byteRowKey =
new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID).getRowKey();
FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(byteRowKey);
assertEquals(CLUSTER, rowKey.getClusterId());
assertEquals(USER, rowKey.getUserId());
assertEquals(FLOW_NAME, rowKey.getFlowName());
assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
byte[] byteRowKeyPrefix =
new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, null).getRowKey();
byte[][] splits =
Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
assertEquals(4, splits.length);
assertEquals(0, splits[3].length);
assertEquals(FLOW_NAME,
Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
verifyRowPrefixBytes(byteRowKeyPrefix);
}
}