|
@@ -20,6 +20,7 @@ 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.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import java.io.IOException;
|
|
@@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
|
|
|
import org.apache.hadoop.hbase.client.Scan;
|
|
|
import org.apache.hadoop.hbase.util.Bytes;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
|
@@ -50,25 +50,28 @@ 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.TimelineMetric.Type;
|
|
|
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
|
|
|
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
|
|
|
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
|
|
|
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
|
|
|
+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.reader.filter.TimelinePrefixFilter;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn;
|
|
|
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.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.Separator;
|
|
|
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
|
|
|
+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;
|
|
@@ -482,7 +485,6 @@ public class TestHBaseTimelineStorage {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
@Test
|
|
|
public void testWriteNullApplicationToHBase() throws Exception {
|
|
|
TimelineEntities te = new TimelineEntities();
|
|
@@ -494,7 +496,7 @@ public class TestHBaseTimelineStorage {
|
|
|
|
|
|
// add the info map in Timeline Entity
|
|
|
Map<String, Object> infoMap = new HashMap<String, Object>();
|
|
|
- infoMap.put("infoMapKey1", "infoMapValue1");
|
|
|
+ infoMap.put("in fo M apK ey1", "infoMapValue1");
|
|
|
infoMap.put("infoMapKey2", 10);
|
|
|
entity.addInfo(infoMap);
|
|
|
|
|
@@ -517,6 +519,7 @@ public class TestHBaseTimelineStorage {
|
|
|
// retrieve the row
|
|
|
Scan scan = new Scan();
|
|
|
scan.setStartRow(Bytes.toBytes(cluster));
|
|
|
+ scan.setStopRow(Bytes.toBytes(cluster + "1"));
|
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
|
ResultScanner resultScanner = new ApplicationTable()
|
|
|
.getResultScanner(c1, conn, scan);
|
|
@@ -626,7 +629,7 @@ public class TestHBaseTimelineStorage {
|
|
|
hbi.start();
|
|
|
String cluster = "cluster_test_write_app";
|
|
|
String user = "user1";
|
|
|
- String flow = "some_flow_name";
|
|
|
+ String flow = "s!ome_f\tlow _n am!e";
|
|
|
String flowVersion = "AB7822C10F1111";
|
|
|
long runid = 1002345678919L;
|
|
|
hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
|
|
@@ -670,7 +673,8 @@ public class TestHBaseTimelineStorage {
|
|
|
assertEquals(cTime, cTime1);
|
|
|
|
|
|
Map<String, Object> infoColumns =
|
|
|
- ApplicationColumnPrefix.INFO.readResults(result);
|
|
|
+ ApplicationColumnPrefix.INFO.readResults(result,
|
|
|
+ StringKeyConverter.getInstance());
|
|
|
assertEquals(infoMap, infoColumns);
|
|
|
|
|
|
// Remember isRelatedTo is of type Map<String, Set<String>>
|
|
@@ -706,11 +710,13 @@ public class TestHBaseTimelineStorage {
|
|
|
|
|
|
// Configuration
|
|
|
Map<String, Object> configColumns =
|
|
|
- ApplicationColumnPrefix.CONFIG.readResults(result);
|
|
|
+ ApplicationColumnPrefix.CONFIG.readResults(result,
|
|
|
+ StringKeyConverter.getInstance());
|
|
|
assertEquals(conf, configColumns);
|
|
|
|
|
|
NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
|
|
|
- ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
|
|
|
+ ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(
|
|
|
+ result, StringKeyConverter.getInstance());
|
|
|
|
|
|
NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
|
|
|
matchMetrics(metricValues, metricMap);
|
|
@@ -868,7 +874,8 @@ public class TestHBaseTimelineStorage {
|
|
|
assertEquals(cTime1, cTime);
|
|
|
|
|
|
Map<String, Object> infoColumns =
|
|
|
- EntityColumnPrefix.INFO.readResults(result);
|
|
|
+ EntityColumnPrefix.INFO.readResults(result,
|
|
|
+ StringKeyConverter.getInstance());
|
|
|
assertEquals(infoMap, infoColumns);
|
|
|
|
|
|
// Remember isRelatedTo is of type Map<String, Set<String>>
|
|
@@ -906,11 +913,12 @@ public class TestHBaseTimelineStorage {
|
|
|
|
|
|
// Configuration
|
|
|
Map<String, Object> configColumns =
|
|
|
- EntityColumnPrefix.CONFIG.readResults(result);
|
|
|
+ EntityColumnPrefix.CONFIG.readResults(result, StringKeyConverter.getInstance());
|
|
|
assertEquals(conf, configColumns);
|
|
|
|
|
|
NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
|
|
|
- EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
|
|
|
+ EntityColumnPrefix.METRIC.readResultsWithTimestamps(
|
|
|
+ result, StringKeyConverter.getInstance());
|
|
|
|
|
|
NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
|
|
|
matchMetrics(metricValues, metricMap);
|
|
@@ -963,7 +971,7 @@ public class TestHBaseTimelineStorage {
|
|
|
}
|
|
|
|
|
|
private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
|
|
|
- String flow, long runid, String appName, TimelineEntity te) {
|
|
|
+ String flow, Long runid, String appName, TimelineEntity te) {
|
|
|
|
|
|
EntityRowKey key = EntityRowKey.parseRowKey(rowKey);
|
|
|
|
|
@@ -978,7 +986,7 @@ public class TestHBaseTimelineStorage {
|
|
|
}
|
|
|
|
|
|
private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
|
|
|
- String user, String flow, long runid, String appName) {
|
|
|
+ String user, String flow, Long runid, String appName) {
|
|
|
|
|
|
ApplicationRowKey key = ApplicationRowKey.parseRowKey(rowKey);
|
|
|
|
|
@@ -995,7 +1003,7 @@ public class TestHBaseTimelineStorage {
|
|
|
TimelineEvent event = new TimelineEvent();
|
|
|
String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
|
|
|
event.setId(eventId);
|
|
|
- long expTs = 1436512802000L;
|
|
|
+ Long expTs = 1436512802000L;
|
|
|
event.setTimestamp(expTs);
|
|
|
String expKey = "foo_event";
|
|
|
Object expVal = "test";
|
|
@@ -1038,20 +1046,18 @@ public class TestHBaseTimelineStorage {
|
|
|
assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
|
|
|
appName));
|
|
|
|
|
|
- Map<?, Object> eventsResult =
|
|
|
- ApplicationColumnPrefix.EVENT.
|
|
|
- readResultsHavingCompoundColumnQualifiers(result);
|
|
|
+ Map<EventColumnName, Object> eventsResult =
|
|
|
+ ApplicationColumnPrefix.EVENT.readResults(result,
|
|
|
+ EventColumnNameConverter.getInstance());
|
|
|
// there should be only one event
|
|
|
assertEquals(1, eventsResult.size());
|
|
|
- for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
|
|
|
+ for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
|
|
|
+ EventColumnName eventColumnName = e.getKey();
|
|
|
// the qualifier is a compound key
|
|
|
// hence match individual values
|
|
|
- byte[][] karr = (byte[][])e.getKey();
|
|
|
- assertEquals(3, karr.length);
|
|
|
- assertEquals(eventId, Bytes.toString(karr[0]));
|
|
|
- assertEquals(
|
|
|
- TimelineStorageUtils.invertLong(expTs), Bytes.toLong(karr[1]));
|
|
|
- assertEquals(expKey, Bytes.toString(karr[2]));
|
|
|
+ assertEquals(eventId, eventColumnName.getId());
|
|
|
+ assertEquals(expTs, eventColumnName.getTimestamp());
|
|
|
+ assertEquals(expKey, eventColumnName.getInfoKey());
|
|
|
Object value = e.getValue();
|
|
|
// there should be only one timestamp and value
|
|
|
assertEquals(expVal, value.toString());
|
|
@@ -1076,7 +1082,7 @@ public class TestHBaseTimelineStorage {
|
|
|
assertEquals(1, events.size());
|
|
|
for (TimelineEvent e : events) {
|
|
|
assertEquals(eventId, e.getId());
|
|
|
- assertEquals(expTs, e.getTimestamp());
|
|
|
+ assertEquals(expTs, Long.valueOf(e.getTimestamp()));
|
|
|
Map<String,Object> info = e.getInfo();
|
|
|
assertEquals(1, info.size());
|
|
|
for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
|
|
@@ -1095,9 +1101,9 @@ public class TestHBaseTimelineStorage {
|
|
|
@Test
|
|
|
public void testEventsWithEmptyInfo() throws IOException {
|
|
|
TimelineEvent event = new TimelineEvent();
|
|
|
- String eventId = "foo_event_id";
|
|
|
+ String eventId = "foo_ev e nt_id";
|
|
|
event.setId(eventId);
|
|
|
- long expTs = 1436512802000L;
|
|
|
+ Long expTs = 1436512802000L;
|
|
|
event.setTimestamp(expTs);
|
|
|
|
|
|
final TimelineEntity entity = new TimelineEntity();
|
|
@@ -1142,21 +1148,19 @@ public class TestHBaseTimelineStorage {
|
|
|
assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
|
|
|
entity));
|
|
|
|
|
|
- Map<?, Object> eventsResult =
|
|
|
- EntityColumnPrefix.EVENT.
|
|
|
- readResultsHavingCompoundColumnQualifiers(result);
|
|
|
+ Map<EventColumnName, Object> eventsResult =
|
|
|
+ EntityColumnPrefix.EVENT.readResults(result,
|
|
|
+ EventColumnNameConverter.getInstance());
|
|
|
// there should be only one event
|
|
|
assertEquals(1, eventsResult.size());
|
|
|
- for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
|
|
|
+ for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
|
|
|
+ EventColumnName eventColumnName = e.getKey();
|
|
|
// the qualifier is a compound key
|
|
|
// hence match individual values
|
|
|
- byte[][] karr = (byte[][])e.getKey();
|
|
|
- assertEquals(3, karr.length);
|
|
|
- assertEquals(eventId, Bytes.toString(karr[0]));
|
|
|
- assertEquals(TimelineStorageUtils.invertLong(expTs),
|
|
|
- Bytes.toLong(karr[1]));
|
|
|
+ assertEquals(eventId, eventColumnName.getId());
|
|
|
+ assertEquals(expTs,eventColumnName.getTimestamp());
|
|
|
// key must be empty
|
|
|
- assertEquals(0, karr[2].length);
|
|
|
+ assertNull(eventColumnName.getInfoKey());
|
|
|
Object value = e.getValue();
|
|
|
// value should be empty
|
|
|
assertEquals("", value.toString());
|
|
@@ -1184,7 +1188,7 @@ public class TestHBaseTimelineStorage {
|
|
|
assertEquals(1, events.size());
|
|
|
for (TimelineEvent e : events) {
|
|
|
assertEquals(eventId, e.getId());
|
|
|
- assertEquals(expTs, e.getTimestamp());
|
|
|
+ assertEquals(expTs, Long.valueOf(e.getTimestamp()));
|
|
|
Map<String,Object> info = e.getInfo();
|
|
|
assertTrue(info == null || info.isEmpty());
|
|
|
}
|
|
@@ -1194,6 +1198,67 @@ public class TestHBaseTimelineStorage {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testEventsEscapeTs() throws IOException {
|
|
|
+ TimelineEvent event = new TimelineEvent();
|
|
|
+ String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
|
|
|
+ event.setId(eventId);
|
|
|
+ long expTs = 1463567041056L;
|
|
|
+ event.setTimestamp(expTs);
|
|
|
+ String expKey = "f==o o_e ve\tnt";
|
|
|
+ Object expVal = "test";
|
|
|
+ event.addInfo(expKey, expVal);
|
|
|
+
|
|
|
+ final TimelineEntity entity = new ApplicationEntity();
|
|
|
+ entity.setId(ApplicationId.newInstance(0, 1).toString());
|
|
|
+ entity.addEvent(event);
|
|
|
+
|
|
|
+ TimelineEntities entities = new TimelineEntities();
|
|
|
+ entities.addEntity(entity);
|
|
|
+
|
|
|
+ HBaseTimelineWriterImpl hbi = null;
|
|
|
+ try {
|
|
|
+ Configuration c1 = util.getConfiguration();
|
|
|
+ hbi = new HBaseTimelineWriterImpl(c1);
|
|
|
+ hbi.init(c1);
|
|
|
+ hbi.start();
|
|
|
+ String cluster = "clus!ter_\ttest_ev ents";
|
|
|
+ String user = "user2";
|
|
|
+ String flow = "other_flow_name";
|
|
|
+ String flowVersion = "1111F01C2287BA";
|
|
|
+ long runid = 1009876543218L;
|
|
|
+ String appName = "application_123465899910_2001";
|
|
|
+ hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
|
|
|
+ hbi.stop();
|
|
|
+
|
|
|
+ // read the timeline entity using the reader this time
|
|
|
+ TimelineEntity e1 = reader.getEntity(
|
|
|
+ new TimelineReaderContext(cluster, user, flow, runid, appName,
|
|
|
+ entity.getType(), entity.getId()),
|
|
|
+ new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
|
|
|
+ assertNotNull(e1);
|
|
|
+ // check the events
|
|
|
+ NavigableSet<TimelineEvent> events = e1.getEvents();
|
|
|
+ // there should be only one event
|
|
|
+ assertEquals(1, events.size());
|
|
|
+ for (TimelineEvent e : events) {
|
|
|
+ assertEquals(eventId, e.getId());
|
|
|
+ assertEquals(expTs, e.getTimestamp());
|
|
|
+ Map<String,Object> info = e.getInfo();
|
|
|
+ assertEquals(1, info.size());
|
|
|
+ for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
|
|
|
+ assertEquals(expKey, infoEntry.getKey());
|
|
|
+ assertEquals(expVal, infoEntry.getValue());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ if (hbi != null) {
|
|
|
+ hbi.stop();
|
|
|
+ hbi.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testNonIntegralMetricValues() throws IOException {
|
|
|
TimelineEntities teApp = new TimelineEntities();
|