|
@@ -18,43 +18,41 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.timelineservice.storage;
|
|
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertNotNull;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+
|
|
|
import java.io.IOException;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
-import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Map.Entry;
|
|
|
import java.util.NavigableMap;
|
|
|
import java.util.Set;
|
|
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
-import static org.junit.Assert.assertNotNull;
|
|
|
-
|
|
|
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.TableName;
|
|
|
import org.apache.hadoop.hbase.client.Connection;
|
|
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
|
|
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.client.Table;
|
|
|
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.api.records.timelineservice.TimelineEvent;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
|
|
|
-import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
|
|
|
-import org.junit.BeforeClass;
|
|
|
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
|
|
|
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
|
|
|
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
|
|
|
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
|
|
|
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
|
|
|
import org.junit.AfterClass;
|
|
|
+import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
/**
|
|
|
- * Unit test HBaseTimelineWriterImpl
|
|
|
- * YARN 3411
|
|
|
- *
|
|
|
* @throws Exception
|
|
|
*/
|
|
|
public class TestHBaseTimelineWriterImpl {
|
|
@@ -69,12 +67,8 @@ public class TestHBaseTimelineWriterImpl {
|
|
|
}
|
|
|
|
|
|
private static void createSchema() throws IOException {
|
|
|
- byte[][] families = new byte[3][];
|
|
|
- families[0] = EntityColumnFamily.INFO.getInBytes();
|
|
|
- families[1] = EntityColumnFamily.CONFIG.getInBytes();
|
|
|
- families[2] = EntityColumnFamily.METRICS.getInBytes();
|
|
|
- TimelineSchemaCreator.createTimelineEntityTable(util.getHBaseAdmin(),
|
|
|
- util.getConfiguration());
|
|
|
+ new EntityTable()
|
|
|
+ .createTable(util.getHBaseAdmin(), util.getConfiguration());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -151,18 +145,15 @@ public class TestHBaseTimelineWriterImpl {
|
|
|
|
|
|
// scan the table and see that entity exists
|
|
|
Scan s = new Scan();
|
|
|
- byte[] startRow = TimelineWriterUtils.getRowKeyPrefix(cluster, user, flow,
|
|
|
- runid, appName);
|
|
|
+ byte[] startRow =
|
|
|
+ EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
|
|
|
s.setStartRow(startRow);
|
|
|
s.setMaxVersions(Integer.MAX_VALUE);
|
|
|
- ResultScanner scanner = null;
|
|
|
- TableName entityTableName = TableName
|
|
|
- .valueOf(TimelineEntitySchemaConstants.DEFAULT_ENTITY_TABLE_NAME);
|
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
|
- Table entityTable = conn.getTable(entityTableName);
|
|
|
+ ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
|
|
|
+
|
|
|
int rowCount = 0;
|
|
|
int colCount = 0;
|
|
|
- scanner = entityTable.getScanner(s);
|
|
|
for (Result result : scanner) {
|
|
|
if (result != null && !result.isEmpty()) {
|
|
|
rowCount++;
|
|
@@ -172,37 +163,77 @@ public class TestHBaseTimelineWriterImpl {
|
|
|
entity));
|
|
|
|
|
|
// check info column family
|
|
|
- NavigableMap<byte[], byte[]> infoValues = result
|
|
|
- .getFamilyMap(EntityColumnFamily.INFO.getInBytes());
|
|
|
- String id1 = TimelineWriterUtils.getValueAsString(
|
|
|
- EntityColumnDetails.ID.getInBytes(), infoValues);
|
|
|
+ String id1 = EntityColumn.ID.readResult(result).toString();
|
|
|
assertEquals(id, id1);
|
|
|
- String type1 = TimelineWriterUtils.getValueAsString(
|
|
|
- EntityColumnDetails.TYPE.getInBytes(), infoValues);
|
|
|
+
|
|
|
+ String type1 = EntityColumn.TYPE.readResult(result).toString();
|
|
|
assertEquals(type, type1);
|
|
|
- Long cTime1 = TimelineWriterUtils.getValueAsLong(
|
|
|
- EntityColumnDetails.CREATED_TIME.getInBytes(), infoValues);
|
|
|
+
|
|
|
+ Number val = (Number) EntityColumn.CREATED_TIME.readResult(result);
|
|
|
+ Long cTime1 = val.longValue();
|
|
|
assertEquals(cTime1, cTime);
|
|
|
- Long mTime1 = TimelineWriterUtils.getValueAsLong(
|
|
|
- EntityColumnDetails.MODIFIED_TIME.getInBytes(), infoValues);
|
|
|
+
|
|
|
+ val = (Number) EntityColumn.MODIFIED_TIME.readResult(result);
|
|
|
+ Long mTime1 = val.longValue();
|
|
|
assertEquals(mTime1, mTime);
|
|
|
- checkRelatedEntities(isRelatedTo, infoValues,
|
|
|
- EntityColumnDetails.PREFIX_IS_RELATED_TO.getInBytes());
|
|
|
- checkRelatedEntities(relatesTo, infoValues,
|
|
|
- EntityColumnDetails.PREFIX_RELATES_TO.getInBytes());
|
|
|
-
|
|
|
- // check config column family
|
|
|
- NavigableMap<byte[], byte[]> configValuesResult = result
|
|
|
- .getFamilyMap(EntityColumnFamily.CONFIG.getInBytes());
|
|
|
- checkConfigs(configValuesResult, conf);
|
|
|
-
|
|
|
- NavigableMap<byte[], byte[]> metricsResult = result
|
|
|
- .getFamilyMap(EntityColumnFamily.METRICS.getInBytes());
|
|
|
- checkMetricsSizeAndKey(metricsResult, metrics);
|
|
|
- List<Cell> metricCells = result.getColumnCells(
|
|
|
- EntityColumnFamily.METRICS.getInBytes(),
|
|
|
- Bytes.toBytes(m1.getId()));
|
|
|
- checkMetricsTimeseries(metricCells, m1);
|
|
|
+
|
|
|
+ // Remember isRelatedTo is of type Map<String, Set<String>>
|
|
|
+ for (String isRelatedToKey : isRelatedTo.keySet()) {
|
|
|
+ Object isRelatedToValue =
|
|
|
+ EntityColumnPrefix.IS_RELATED_TO.readResult(result,
|
|
|
+ isRelatedToKey);
|
|
|
+ String compoundValue = isRelatedToValue.toString();
|
|
|
+ // id7?id9?id6
|
|
|
+ Set<String> isRelatedToValues =
|
|
|
+ new HashSet<String>(
|
|
|
+ Separator.VALUES.splitEncoded(compoundValue));
|
|
|
+ assertEquals(isRelatedTo.get(isRelatedToKey).size(),
|
|
|
+ isRelatedToValues.size());
|
|
|
+ for (String v : isRelatedTo.get(isRelatedToKey)) {
|
|
|
+ assertTrue(isRelatedToValues.contains(v));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // RelatesTo
|
|
|
+ for (String relatesToKey : relatesTo.keySet()) {
|
|
|
+ String compoundValue =
|
|
|
+ EntityColumnPrefix.RELATES_TO.readResult(result, relatesToKey)
|
|
|
+ .toString();
|
|
|
+ // id3?id4?id5
|
|
|
+ Set<String> relatesToValues =
|
|
|
+ new HashSet<String>(
|
|
|
+ Separator.VALUES.splitEncoded(compoundValue));
|
|
|
+ assertEquals(relatesTo.get(relatesToKey).size(),
|
|
|
+ relatesToValues.size());
|
|
|
+ for (String v : relatesTo.get(relatesToKey)) {
|
|
|
+ assertTrue(relatesToValues.contains(v));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Configuration
|
|
|
+ Map<String, Object> configColumns =
|
|
|
+ EntityColumnPrefix.CONFIG.readResults(result);
|
|
|
+ assertEquals(conf.size(), configColumns.size());
|
|
|
+ for (String configItem : conf.keySet()) {
|
|
|
+ assertEquals(conf.get(configItem), configColumns.get(configItem));
|
|
|
+ }
|
|
|
+
|
|
|
+ NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
|
|
|
+ EntityColumnPrefix.METRIC.readTimeseriesResults(result);
|
|
|
+
|
|
|
+ NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
|
|
|
+ // We got metrics back
|
|
|
+ assertNotNull(metricMap);
|
|
|
+ // Same number of metrics as we wrote
|
|
|
+ assertEquals(metricValues.entrySet().size(), metricMap.entrySet()
|
|
|
+ .size());
|
|
|
+
|
|
|
+ // Iterate over original metrics and confirm that they are present
|
|
|
+ // here.
|
|
|
+ for (Entry<Long, Number> metricEntry : metricValues.entrySet()) {
|
|
|
+ assertEquals(metricEntry.getValue(),
|
|
|
+ metricMap.get(metricEntry.getKey()));
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
assertEquals(1, rowCount);
|
|
@@ -212,80 +243,77 @@ public class TestHBaseTimelineWriterImpl {
|
|
|
hbi.stop();
|
|
|
hbi.close();
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- private void checkMetricsTimeseries(List<Cell> metricCells,
|
|
|
- TimelineMetric m1) throws IOException {
|
|
|
- Map<Long, Number> timeseries = m1.getValues();
|
|
|
- assertEquals(timeseries.size(), metricCells.size());
|
|
|
- for (Cell c1 : metricCells) {
|
|
|
- assertTrue(timeseries.containsKey(c1.getTimestamp()));
|
|
|
- assertEquals(GenericObjectMapper.read(CellUtil.cloneValue(c1)),
|
|
|
- timeseries.get(c1.getTimestamp()));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void checkMetricsSizeAndKey(
|
|
|
- NavigableMap<byte[], byte[]> metricsResult, Set<TimelineMetric> metrics) {
|
|
|
- assertEquals(metrics.size(), metricsResult.size());
|
|
|
- for (TimelineMetric m1 : metrics) {
|
|
|
- byte[] key = Bytes.toBytes(m1.getId());
|
|
|
- assertTrue(metricsResult.containsKey(key));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void checkConfigs(NavigableMap<byte[], byte[]> configValuesResult,
|
|
|
- Map<String, String> conf) throws IOException {
|
|
|
-
|
|
|
- assertEquals(conf.size(), configValuesResult.size());
|
|
|
- byte[] columnName;
|
|
|
- for (String key : conf.keySet()) {
|
|
|
- columnName = Bytes.toBytes(key);
|
|
|
- assertTrue(configValuesResult.containsKey(columnName));
|
|
|
- byte[] value = configValuesResult.get(columnName);
|
|
|
- assertNotNull(value);
|
|
|
- assertEquals(conf.get(key), GenericObjectMapper.read(value));
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- private void checkRelatedEntities(Map<String, Set<String>> isRelatedTo,
|
|
|
- NavigableMap<byte[], byte[]> infoValues, byte[] columnPrefix)
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- for (String key : isRelatedTo.keySet()) {
|
|
|
- byte[] columnName = TimelineWriterUtils.join(
|
|
|
- TimelineEntitySchemaConstants.ROW_KEY_SEPARATOR_BYTES, columnPrefix,
|
|
|
- Bytes.toBytes(key));
|
|
|
-
|
|
|
- byte[] value = infoValues.get(columnName);
|
|
|
- assertNotNull(value);
|
|
|
- String isRelatedToEntities = GenericObjectMapper.read(value).toString();
|
|
|
- assertNotNull(isRelatedToEntities);
|
|
|
- assertEquals(
|
|
|
- TimelineWriterUtils.getValueAsString(
|
|
|
- TimelineEntitySchemaConstants.ROW_KEY_SEPARATOR,
|
|
|
- isRelatedTo.get(key)), isRelatedToEntities);
|
|
|
- }
|
|
|
+ // Somewhat of a hack, not a separate test in order not to have to deal with
|
|
|
+ // test case order exectution.
|
|
|
+ testAdditionalEntity();
|
|
|
}
|
|
|
|
|
|
private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
|
|
|
String flow, Long runid, String appName, TimelineEntity te) {
|
|
|
|
|
|
- byte[][] rowKeyComponents = TimelineWriterUtils.split(rowKey,
|
|
|
- TimelineEntitySchemaConstants.ROW_KEY_SEPARATOR_BYTES);
|
|
|
+ byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey, -1);
|
|
|
|
|
|
assertTrue(rowKeyComponents.length == 7);
|
|
|
assertEquals(user, Bytes.toString(rowKeyComponents[0]));
|
|
|
assertEquals(cluster, Bytes.toString(rowKeyComponents[1]));
|
|
|
assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
|
|
|
- assertEquals(TimelineWriterUtils.encodeRunId(runid),
|
|
|
- Bytes.toLong(rowKeyComponents[3]));
|
|
|
- assertEquals(TimelineWriterUtils.cleanse(appName), Bytes.toString(rowKeyComponents[4]));
|
|
|
+ assertEquals(EntityRowKey.invert(runid), Bytes.toLong(rowKeyComponents[3]));
|
|
|
+ assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
|
|
|
assertEquals(te.getType(), Bytes.toString(rowKeyComponents[5]));
|
|
|
assertEquals(te.getId(), Bytes.toString(rowKeyComponents[6]));
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
+ private void testAdditionalEntity() throws IOException {
|
|
|
+ TimelineEvent event = new TimelineEvent();
|
|
|
+ event.setId("foo_event_id");
|
|
|
+ event.setTimestamp(System.currentTimeMillis());
|
|
|
+ event.addInfo("foo_event", "test");
|
|
|
+
|
|
|
+ final TimelineEntity entity = new TimelineEntity();
|
|
|
+ entity.setId("attempt_1329348432655_0001_m_000008_18");
|
|
|
+ entity.setType("FOO_ATTEMPT");
|
|
|
+
|
|
|
+ TimelineEntities entities = new TimelineEntities();
|
|
|
+ entities.addEntity(entity);
|
|
|
+
|
|
|
+ HBaseTimelineWriterImpl hbi = null;
|
|
|
+ try {
|
|
|
+ Configuration c1 = util.getConfiguration();
|
|
|
+ hbi = new HBaseTimelineWriterImpl(c1);
|
|
|
+ hbi.init(c1);
|
|
|
+ String cluster = "cluster2";
|
|
|
+ String user = "user2";
|
|
|
+ String flow = "other_flow_name";
|
|
|
+ String flowVersion = "1111F01C2287BA";
|
|
|
+ long runid = 1009876543218L;
|
|
|
+ String appName = "some app name";
|
|
|
+ hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
|
|
|
+ hbi.stop();
|
|
|
+ // scan the table and see that entity exists
|
|
|
+ Scan s = new Scan();
|
|
|
+ byte[] startRow =
|
|
|
+ EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
|
|
|
+ s.setStartRow(startRow);
|
|
|
+ s.setMaxVersions(Integer.MAX_VALUE);
|
|
|
+ Connection conn = ConnectionFactory.createConnection(c1);
|
|
|
+ ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
|
|
|
+
|
|
|
+ int rowCount = 0;
|
|
|
+ for (Result result : scanner) {
|
|
|
+ if (result != null && !result.isEmpty()) {
|
|
|
+ rowCount++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ assertEquals(1, rowCount);
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ hbi.stop();
|
|
|
+ hbi.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@AfterClass
|
|
|
public static void tearDownAfterClass() throws Exception {
|
|
|
util.shutdownMiniCluster();
|