YARN-5170. Eliminate singleton converters and static method access. (Joep Rottinghuis via Varun Saxena)
This commit is contained in:
parent
3832795e3c
commit
892b193bd7
|
@ -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()) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
}
|
|
@ -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() {
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue