diff --git hadoop-project/pom.xml hadoop-project/pom.xml
index 01184c9..1621b4e 100644
--- hadoop-project/pom.xml
+++ hadoop-project/pom.xml
@@ -325,6 +325,24 @@
${project.version}
+
+ org.apache.hadoop
+ hadoop-yarn-server-timelineservice-hbase-schema
+ ${project.version}
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-timelineservice-hbase-client
+ ${project.version}
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-timelineservice-hbase-server
+ ${project.version}
+
+
org.apache.hadoop
hadoop-yarn-applications-distributedshell
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
index de99b95..2d9e651 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
@@ -16,8 +16,8 @@
limitations under the License.
-->
hadoop-yarn-server
@@ -61,7 +61,31 @@
org.apache.hadoop
- hadoop-yarn-server-timelineservice-hbase
+ hadoop-yarn-server-timelineservice-hbase-schema
+ test
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-timelineservice-hbase-client
+ test
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-timelineservice-hbase-server
test
@@ -340,7 +364,7 @@
org.codehaus.mojo
findbugs-maven-plugin
-
+
true
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index 38865f1..9fe8fc8 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -54,7 +54,7 @@
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest;
import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
import org.junit.After;
@@ -81,7 +81,7 @@
private static HBaseTestingUtility util;
private static long ts = System.currentTimeMillis();
private static long dayTs =
- HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts);
+ HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(ts);
@BeforeClass
public static void setup() throws Exception {
@@ -1017,7 +1017,7 @@ public void testGetFlows() throws Exception {
new String[] {"flow1"});
long firstFlowActivity =
- HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(1425016501000L);
+ HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(1425016501000L);
DateFormat fmt = TimelineReaderWebServices.DATE_FORMAT.get();
uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
index 1aff2e0..59423e9 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
@@ -66,7 +66,9 @@
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.application.ApplicationTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefixRWHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnRWHelper;
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;
@@ -168,7 +170,7 @@ public void testWriteNullApplicationToHBase() throws Exception {
scan.setStartRow(Bytes.toBytes(cluster));
scan.setStopRow(Bytes.toBytes(cluster + "1"));
Connection conn = ConnectionFactory.createConnection(c1);
- ResultScanner resultScanner = new ApplicationTable()
+ ResultScanner resultScanner = new ApplicationTableRW()
.getResultScanner(c1, conn, scan);
assertTrue(resultScanner != null);
@@ -302,7 +304,7 @@ public void testWriteApplicationToHBase() throws Exception {
Get get = new Get(rowKey);
get.setMaxVersions(Integer.MAX_VALUE);
Connection conn = ConnectionFactory.createConnection(c1);
- Result result = new ApplicationTable().getResult(c1, conn, get);
+ Result result = new ApplicationTableRW().getResult(c1, conn, get);
assertTrue(result != null);
assertEquals(17, result.size());
@@ -313,24 +315,23 @@ public void testWriteApplicationToHBase() throws Exception {
appId));
// check info column family
- String id1 = ApplicationColumn.ID.readResult(result).toString();
+ String id1 = ColumnRWHelper.readResult(result, ApplicationColumn.ID)
+ .toString();
assertEquals(appId, id1);
- Long cTime1 =
- (Long) ApplicationColumn.CREATED_TIME.readResult(result);
+ Long cTime1 = (Long) ColumnRWHelper.readResult(result,
+ ApplicationColumn.CREATED_TIME);
assertEquals(cTime, cTime1);
- Map infoColumns =
- ApplicationColumnPrefix.INFO.readResults(result,
- new StringKeyConverter());
+ Map infoColumns = ColumnPrefixRWHelper.readResults(result,
+ ApplicationColumnPrefix.INFO, new StringKeyConverter());
assertEquals(infoMap, infoColumns);
// Remember isRelatedTo is of type Map>
for (Map.Entry> isRelatedToEntry : isRelatedTo
.entrySet()) {
- Object isRelatedToValue =
- ApplicationColumnPrefix.IS_RELATED_TO.readResult(result,
- isRelatedToEntry.getKey());
+ Object isRelatedToValue = ColumnPrefixRWHelper.readResult(result,
+ ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToEntry.getKey());
String compoundValue = isRelatedToValue.toString();
// id7?id9?id6
Set isRelatedToValues =
@@ -345,9 +346,9 @@ public void testWriteApplicationToHBase() throws Exception {
// RelatesTo
for (Map.Entry> relatesToEntry : relatesTo
.entrySet()) {
- String compoundValue =
- ApplicationColumnPrefix.RELATES_TO.readResult(result,
- relatesToEntry.getKey()).toString();
+ String compoundValue = ColumnPrefixRWHelper.readResult(result,
+ ApplicationColumnPrefix.RELATES_TO, relatesToEntry.getKey())
+ .toString();
// id3?id4?id5
Set relatesToValues =
new HashSet(Separator.VALUES.splitEncoded(compoundValue));
@@ -360,14 +361,13 @@ public void testWriteApplicationToHBase() throws Exception {
KeyConverter stringKeyConverter = new StringKeyConverter();
// Configuration
- Map configColumns =
- ApplicationColumnPrefix.CONFIG
- .readResults(result, stringKeyConverter);
+ Map configColumns = ColumnPrefixRWHelper.readResults(
+ result, ApplicationColumnPrefix.CONFIG, stringKeyConverter);
assertEquals(conf, configColumns);
NavigableMap> metricsResult =
- ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result,
- stringKeyConverter);
+ ColumnPrefixRWHelper.readResultsWithTimestamps(result,
+ ApplicationColumnPrefix.METRIC, stringKeyConverter);
NavigableMap metricMap = metricsResult.get(m1.getId());
matchMetrics(metricValues, metricMap);
@@ -517,7 +517,7 @@ public void testEvents() throws IOException {
Get get = new Get(rowKey);
get.setMaxVersions(Integer.MAX_VALUE);
Connection conn = ConnectionFactory.createConnection(c1);
- Result result = new ApplicationTable().getResult(c1, conn, get);
+ Result result = new ApplicationTableRW().getResult(c1, conn, get);
assertTrue(result != null);
@@ -527,8 +527,8 @@ public void testEvents() throws IOException {
appName));
Map eventsResult =
- ApplicationColumnPrefix.EVENT.readResults(result,
- new EventColumnNameConverter());
+ ColumnPrefixRWHelper.readResults(result,
+ ApplicationColumnPrefix.EVENT, new EventColumnNameConverter());
// there should be only one event
assertEquals(1, eventsResult.size());
for (Map.Entry e : eventsResult.entrySet()) {
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
index c731631..6e3b6bb 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
@@ -60,6 +60,8 @@
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.common.ColumnPrefixRWHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnRWHelper;
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;
@@ -70,7 +72,7 @@
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.server.timelineservice.storage.entity.EntityTableRW;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -211,7 +213,7 @@ public void testWriteEntityToHBase() throws Exception {
s.setStartRow(startRow);
s.setMaxVersions(Integer.MAX_VALUE);
Connection conn = ConnectionFactory.createConnection(c1);
- ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
+ ResultScanner scanner = new EntityTableRW().getResultScanner(c1, conn, s);
int rowCount = 0;
int colCount = 0;
@@ -225,26 +227,27 @@ public void testWriteEntityToHBase() throws Exception {
entity));
// check info column family
- String id1 = EntityColumn.ID.readResult(result).toString();
+ String id1 = ColumnRWHelper.readResult(result, EntityColumn.ID)
+ .toString();
assertEquals(id, id1);
- String type1 = EntityColumn.TYPE.readResult(result).toString();
+ String type1 = ColumnRWHelper.readResult(result, EntityColumn.TYPE)
+ .toString();
assertEquals(type, type1);
- Long cTime1 = (Long) EntityColumn.CREATED_TIME.readResult(result);
+ Long cTime1 = (Long)
+ ColumnRWHelper.readResult(result, EntityColumn.CREATED_TIME);
assertEquals(cTime1, cTime);
- Map infoColumns =
- EntityColumnPrefix.INFO.readResults(result,
- new StringKeyConverter());
+ Map infoColumns = ColumnPrefixRWHelper.readResults(
+ result, EntityColumnPrefix.INFO, new StringKeyConverter());
assertEquals(infoMap, infoColumns);
// Remember isRelatedTo is of type Map>
for (Map.Entry> isRelatedToEntry : isRelatedTo
.entrySet()) {
- Object isRelatedToValue =
- EntityColumnPrefix.IS_RELATED_TO.readResult(result,
- isRelatedToEntry.getKey());
+ Object isRelatedToValue = ColumnPrefixRWHelper.readResult(result,
+ EntityColumnPrefix.IS_RELATED_TO, isRelatedToEntry.getKey());
String compoundValue = isRelatedToValue.toString();
// id7?id9?id6
Set isRelatedToValues =
@@ -260,8 +263,9 @@ public void testWriteEntityToHBase() throws Exception {
// RelatesTo
for (Map.Entry> relatesToEntry : relatesTo
.entrySet()) {
- String compoundValue = EntityColumnPrefix.RELATES_TO
- .readResult(result, relatesToEntry.getKey()).toString();
+ String compoundValue = ColumnPrefixRWHelper.readResult(result,
+ EntityColumnPrefix.RELATES_TO, relatesToEntry.getKey())
+ .toString();
// id3?id4?id5
Set relatesToValues =
new HashSet(
@@ -274,13 +278,13 @@ public void testWriteEntityToHBase() throws Exception {
}
// Configuration
- Map configColumns =
- EntityColumnPrefix.CONFIG.readResults(result, stringKeyConverter);
+ Map configColumns = ColumnPrefixRWHelper.readResults(
+ result, EntityColumnPrefix.CONFIG, stringKeyConverter);
assertEquals(conf, configColumns);
NavigableMap> metricsResult =
- EntityColumnPrefix.METRIC.readResultsWithTimestamps(result,
- stringKeyConverter);
+ ColumnPrefixRWHelper.readResultsWithTimestamps(result,
+ EntityColumnPrefix.METRIC, stringKeyConverter);
NavigableMap metricMap = metricsResult.get(m1.getId());
matchMetrics(metricValues, metricMap);
@@ -407,7 +411,7 @@ public void testEventsWithEmptyInfo() throws IOException {
s.setStartRow(startRow);
s.addFamily(EntityColumnFamily.INFO.getBytes());
Connection conn = ConnectionFactory.createConnection(c1);
- ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
+ ResultScanner scanner = new EntityTableRW().getResultScanner(c1, conn, s);
int rowCount = 0;
for (Result result : scanner) {
@@ -420,8 +424,8 @@ public void testEventsWithEmptyInfo() throws IOException {
entity));
Map eventsResult =
- EntityColumnPrefix.EVENT.readResults(result,
- new EventColumnNameConverter());
+ ColumnPrefixRWHelper.readResults(result,
+ EntityColumnPrefix.EVENT, new EventColumnNameConverter());
// there should be only one event
assertEquals(1, eventsResult.size());
for (Map.Entry e : eventsResult.entrySet()) {
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageSchema.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageSchema.java
index 0dcd171..9505a16 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageSchema.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageSchema.java
@@ -21,6 +21,9 @@
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTableRW;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -35,10 +38,6 @@
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
-
/**
* Unit tests for checking different schema prefixes.
*/
@@ -61,22 +60,22 @@ public void createWithDefaultPrefix() throws IOException {
conn = ConnectionFactory.createConnection(hbaseConf);
Admin admin = conn.getAdmin();
- TableName entityTableName = BaseTable.getTableName(hbaseConf,
- EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME);
+ TableName entityTableName = BaseTableRW.getTableName(hbaseConf,
+ EntityTableRW.TABLE_NAME_CONF_NAME, EntityTableRW.DEFAULT_TABLE_NAME);
assertTrue(admin.tableExists(entityTableName));
assertTrue(entityTableName.getNameAsString().startsWith(
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX));
- Table entityTable = conn.getTable(BaseTable.getTableName(hbaseConf,
- EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME));
+ Table entityTable = conn.getTable(BaseTableRW.getTableName(hbaseConf,
+ EntityTableRW.TABLE_NAME_CONF_NAME, EntityTableRW.DEFAULT_TABLE_NAME));
assertNotNull(entityTable);
- TableName flowRunTableName = BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME);
+ TableName flowRunTableName = BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME);
assertTrue(admin.tableExists(flowRunTableName));
assertTrue(flowRunTableName.getNameAsString().startsWith(
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX));
- Table flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table flowRunTable = conn.getTable(BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
assertNotNull(flowRunTable);
}
@@ -91,20 +90,20 @@ public void createWithSetPrefix() throws IOException {
conn = ConnectionFactory.createConnection(hbaseConf);
Admin admin = conn.getAdmin();
- TableName entityTableName = BaseTable.getTableName(hbaseConf,
- EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME);
+ TableName entityTableName = BaseTableRW.getTableName(hbaseConf,
+ EntityTableRW.TABLE_NAME_CONF_NAME, EntityTableRW.DEFAULT_TABLE_NAME);
assertTrue(admin.tableExists(entityTableName));
assertTrue(entityTableName.getNameAsString().startsWith(prefix));
- Table entityTable = conn.getTable(BaseTable.getTableName(hbaseConf,
- EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME));
+ Table entityTable = conn.getTable(BaseTableRW.getTableName(hbaseConf,
+ EntityTableRW.TABLE_NAME_CONF_NAME, EntityTableRW.DEFAULT_TABLE_NAME));
assertNotNull(entityTable);
- TableName flowRunTableName = BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME);
+ TableName flowRunTableName = BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME);
assertTrue(admin.tableExists(flowRunTableName));
assertTrue(flowRunTableName.getNameAsString().startsWith(prefix));
- Table flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table flowRunTable = conn.getTable(BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
assertNotNull(flowRunTable);
// create another set with a diff prefix
@@ -114,20 +113,20 @@ public void createWithSetPrefix() throws IOException {
hbaseConf.set(YarnConfiguration.TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME,
prefix);
DataGeneratorForTest.createSchema(hbaseConf);
- entityTableName = BaseTable.getTableName(hbaseConf,
- EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME);
+ entityTableName = BaseTableRW.getTableName(hbaseConf,
+ EntityTableRW.TABLE_NAME_CONF_NAME, EntityTableRW.DEFAULT_TABLE_NAME);
assertTrue(admin.tableExists(entityTableName));
assertTrue(entityTableName.getNameAsString().startsWith(prefix));
- entityTable = conn.getTable(BaseTable.getTableName(hbaseConf,
- EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME));
+ entityTable = conn.getTable(BaseTableRW.getTableName(hbaseConf,
+ EntityTableRW.TABLE_NAME_CONF_NAME, EntityTableRW.DEFAULT_TABLE_NAME));
assertNotNull(entityTable);
- flowRunTableName = BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME);
+ flowRunTableName = BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME);
assertTrue(admin.tableExists(flowRunTableName));
assertTrue(flowRunTableName.getNameAsString().startsWith(prefix));
- flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ flowRunTable = conn.getTable(BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
assertNotNull(flowRunTable);
hbaseConf
.unset(YarnConfiguration.TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME);
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
index c21e874..bdca5da 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -50,9 +50,9 @@
import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest;
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.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -152,8 +152,8 @@ public void testWriteFlowRunMinMax() throws Exception {
Connection conn = ConnectionFactory.createConnection(c1);
// check in flow activity table
Table table1 = conn.getTable(
- BaseTable.getTableName(c1, FlowActivityTable.TABLE_NAME_CONF_NAME,
- FlowActivityTable.DEFAULT_TABLE_NAME));
+ BaseTableRW.getTableName(c1, FlowActivityTableRW.TABLE_NAME_CONF_NAME,
+ FlowActivityTableRW.DEFAULT_TABLE_NAME));
byte[] startRow =
new FlowActivityRowKey(cluster, minStartTs, user, flow).getRowKey();
Get g = new Get(startRow);
@@ -169,7 +169,7 @@ public void testWriteFlowRunMinMax() throws Exception {
assertEquals(cluster, flowActivityRowKey.getClusterId());
assertEquals(user, flowActivityRowKey.getUserId());
assertEquals(flow, flowActivityRowKey.getFlowName());
- Long dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(minStartTs);
+ Long dayTs = HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(minStartTs);
assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
assertEquals(1, values.size());
checkFlowActivityRunId(runid, flowVersion, values);
@@ -284,8 +284,8 @@ private void checkFlowActivityTable(String cluster, String user, String flow,
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
Table table1 = conn.getTable(
- BaseTable.getTableName(c1, FlowActivityTable.TABLE_NAME_CONF_NAME,
- FlowActivityTable.DEFAULT_TABLE_NAME));
+ BaseTableRW.getTableName(c1, FlowActivityTableRW.TABLE_NAME_CONF_NAME,
+ FlowActivityTableRW.DEFAULT_TABLE_NAME));
ResultScanner scanner = table1.getScanner(s);
int rowCount = 0;
for (Result result : scanner) {
@@ -301,7 +301,7 @@ private void checkFlowActivityTable(String cluster, String user, String flow,
assertEquals(cluster, flowActivityRowKey.getClusterId());
assertEquals(user, flowActivityRowKey.getUserId());
assertEquals(flow, flowActivityRowKey.getFlowName());
- Long dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(
+ Long dayTs = HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(
appCreatedTime);
assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
assertEquals(1, values.size());
@@ -387,7 +387,7 @@ public void testFlowActivityTableOneFlowMultipleRunIds() throws IOException {
assertEquals(user, flowActivity.getUser());
assertEquals(flow, flowActivity.getFlowName());
long dayTs =
- HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
+ HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(appCreatedTime);
assertEquals(dayTs, flowActivity.getDate().getTime());
Set flowRuns = flowActivity.getFlowRuns();
assertEquals(3, flowRuns.size());
@@ -428,8 +428,8 @@ private void checkFlowActivityTableSeveralRuns(String cluster, String user,
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
Table table1 = conn.getTable(
- BaseTable.getTableName(c1, FlowActivityTable.TABLE_NAME_CONF_NAME,
- FlowActivityTable.DEFAULT_TABLE_NAME));
+ BaseTableRW.getTableName(c1, FlowActivityTableRW.TABLE_NAME_CONF_NAME,
+ FlowActivityTableRW.DEFAULT_TABLE_NAME));
ResultScanner scanner = table1.getScanner(s);
int rowCount = 0;
for (Result result : scanner) {
@@ -442,7 +442,7 @@ private void checkFlowActivityTableSeveralRuns(String cluster, String user,
assertEquals(cluster, flowActivityRowKey.getClusterId());
assertEquals(user, flowActivityRowKey.getUserId());
assertEquals(flow, flowActivityRowKey.getFlowName());
- Long dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(
+ Long dayTs = HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(
appCreatedTime);
assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
index 2a101cf..95dcb54 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -60,9 +60,9 @@
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.TimelineReader.Field;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTableRW;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -89,8 +89,8 @@ public static void setupBeforeClass() throws Exception {
@Test
public void checkCoProcessorOff() throws IOException, InterruptedException {
Configuration hbaseConf = util.getConfiguration();
- TableName table = BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME);
+ TableName table = BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME);
Connection conn = null;
conn = ConnectionFactory.createConnection(hbaseConf);
Admin admin = conn.getAdmin();
@@ -104,9 +104,9 @@ public void checkCoProcessorOff() throws IOException, InterruptedException {
checkCoprocessorExists(table, true);
}
- table = BaseTable.getTableName(hbaseConf,
- FlowActivityTable.TABLE_NAME_CONF_NAME,
- FlowActivityTable.DEFAULT_TABLE_NAME);
+ table = BaseTableRW.getTableName(hbaseConf,
+ FlowActivityTableRW.TABLE_NAME_CONF_NAME,
+ FlowActivityTableRW.DEFAULT_TABLE_NAME);
if (admin.tableExists(table)) {
// check the regions.
// check in flow activity table
@@ -114,8 +114,8 @@ public void checkCoProcessorOff() throws IOException, InterruptedException {
checkCoprocessorExists(table, false);
}
- table = BaseTable.getTableName(hbaseConf, EntityTable.TABLE_NAME_CONF_NAME,
- EntityTable.DEFAULT_TABLE_NAME);
+ table = BaseTableRW.getTableName(hbaseConf, EntityTableRW.TABLE_NAME_CONF_NAME,
+ EntityTableRW.DEFAULT_TABLE_NAME);
if (admin.tableExists(table)) {
// check the regions.
// check in entity run table
@@ -215,8 +215,8 @@ public void testWriteFlowRunMinMax() throws Exception {
Connection conn = ConnectionFactory.createConnection(c1);
// check in flow run table
- Table table1 = conn.getTable(BaseTable.getTableName(c1,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table table1 = conn.getTable(BaseTableRW.getTableName(c1,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
// scan the table and see that we get back the right min and max
// timestamps
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
@@ -367,8 +367,8 @@ void checkFlowRunTableBatchLimit(String cluster, String user, String flow,
.getRowKey();
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
- Table table1 = conn.getTable(BaseTable.getTableName(c1,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table table1 = conn.getTable(BaseTableRW.getTableName(c1,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
ResultScanner scanner = table1.getScanner(s);
int loopCount = 0;
@@ -512,8 +512,8 @@ private void checkFlowRunTable(String cluster, String user, String flow,
new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
- Table table1 = conn.getTable(BaseTable.getTableName(c1,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table table1 = conn.getTable(BaseTableRW.getTableName(c1,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
ResultScanner scanner = table1.getScanner(s);
int rowCount = 0;
@@ -777,8 +777,8 @@ private void checkMinMaxFlush(Configuration c1, long minTS, long startTs,
boolean checkMax) throws IOException {
Connection conn = ConnectionFactory.createConnection(c1);
// check in flow run table
- Table table1 = conn.getTable(BaseTable.getTableName(c1,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table table1 = conn.getTable(BaseTableRW.getTableName(c1,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
// scan the table and see that we get back the right min and max
// timestamps
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
index 7f783b8..30496e4 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
@@ -52,9 +52,9 @@
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest;
import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
import org.junit.AfterClass;
@@ -105,8 +105,8 @@ public void testWriteNonNumericData() throws Exception {
Configuration hbaseConf = util.getConfiguration();
Connection conn = null;
conn = ConnectionFactory.createConnection(hbaseConf);
- Table flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table flowRunTable = conn.getTable(BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
flowRunTable.put(p);
Get g = new Get(rowKeyBytes);
@@ -154,8 +154,8 @@ public void testWriteScanBatchLimit() throws Exception {
Configuration hbaseConf = util.getConfiguration();
Connection conn = null;
conn = ConnectionFactory.createConnection(hbaseConf);
- Table flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table flowRunTable = conn.getTable(BaseTableRW.getTableName(hbaseConf,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
flowRunTable.put(p);
String rowKey2 = "nonNumericRowKey2";
@@ -316,10 +316,10 @@ public void testWriteFlowRunCompaction() throws Exception {
// check in flow run table
HRegionServer server = util.getRSForFirstRegionInTable(
- BaseTable.getTableName(c1, FlowRunTable.TABLE_NAME_CONF_NAME,
- FlowRunTable.DEFAULT_TABLE_NAME));
- List regions = server.getOnlineRegions(BaseTable.getTableName(c1,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ BaseTableRW.getTableName(c1, FlowRunTableRW.TABLE_NAME_CONF_NAME,
+ FlowRunTableRW.DEFAULT_TABLE_NAME));
+ List regions = server.getOnlineRegions(BaseTableRW.getTableName(c1,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
assertTrue("Didn't find any regions for primary table!",
regions.size() > 0);
// flush and compact all the regions of the primary table
@@ -344,8 +344,8 @@ private void checkFlowRunTable(String cluster, String user, String flow,
new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
s.setStopRow(stopRow);
Connection conn = ConnectionFactory.createConnection(c1);
- Table table1 = conn.getTable(BaseTable.getTableName(c1,
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+ Table table1 = conn.getTable(BaseTableRW.getTableName(c1,
+ FlowRunTableRW.TABLE_NAME_CONF_NAME, FlowRunTableRW.DEFAULT_TABLE_NAME));
ResultScanner scanner = table1.getScanner(s);
int rowCount = 0;
@@ -412,7 +412,7 @@ public void checkProcessSummationMoreCellsSumFinal2()
tags.add(t);
byte[] tagByteArray = Tag.fromList(tags);
// create a cell with a VERY old timestamp and attribute SUM_FINAL
- Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily,
+ Cell c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily,
aQualifier, cell1Ts, Bytes.toBytes(cellValue1), tagByteArray);
currentColumnCells.add(c1);
@@ -422,7 +422,7 @@ public void checkProcessSummationMoreCellsSumFinal2()
tags.add(t);
tagByteArray = Tag.fromList(tags);
// create a cell with a recent timestamp and attribute SUM_FINAL
- Cell c2 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily,
+ Cell c2 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily,
aQualifier, cell2Ts, Bytes.toBytes(cellValue2), tagByteArray);
currentColumnCells.add(c2);
@@ -432,7 +432,7 @@ public void checkProcessSummationMoreCellsSumFinal2()
tags.add(t);
tagByteArray = Tag.fromList(tags);
// create a cell with a VERY old timestamp but has attribute SUM
- Cell c3 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily,
+ Cell c3 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily,
aQualifier, cell3Ts, Bytes.toBytes(cellValue3), tagByteArray);
currentColumnCells.add(c3);
@@ -442,7 +442,7 @@ public void checkProcessSummationMoreCellsSumFinal2()
tags.add(t);
tagByteArray = Tag.fromList(tags);
// create a cell with a VERY old timestamp but has attribute SUM
- Cell c4 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily,
+ Cell c4 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily,
aQualifier, cell4Ts, Bytes.toBytes(cellValue4), tagByteArray);
currentColumnCells.add(c4);
@@ -512,7 +512,7 @@ public void checkProcessSummationMoreCellsSumFinalMany() throws IOException {
tags.add(t);
byte[] tagByteArray = Tag.fromList(tags);
// create a cell with a VERY old timestamp and attribute SUM_FINAL
- c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+ c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily, aQualifier,
cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray);
currentColumnCells.add(c1);
cellTsFinal++;
@@ -526,7 +526,7 @@ public void checkProcessSummationMoreCellsSumFinalMany() throws IOException {
tags.add(t);
byte[] tagByteArray = Tag.fromList(tags);
// create a cell with attribute SUM
- c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+ c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily, aQualifier,
cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray);
currentColumnCells.add(c1);
cellTsNotFinal++;
@@ -603,7 +603,7 @@ public void checkProcessSummationMoreCellsSumFinalVariedTags()
tags.add(t);
byte[] tagByteArray = Tag.fromList(tags);
// create a cell with a VERY old timestamp and attribute SUM_FINAL
- c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+ c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily, aQualifier,
cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray);
currentColumnCells.add(c1);
cellTsFinal++;
@@ -617,7 +617,7 @@ public void checkProcessSummationMoreCellsSumFinalVariedTags()
tags.add(t);
byte[] tagByteArray = Tag.fromList(tags);
// create a cell with a VERY old timestamp and attribute SUM_FINAL
- c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+ c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily, aQualifier,
cellTsFinalNotExpire, Bytes.toBytes(cellValueFinal), tagByteArray);
currentColumnCells.add(c1);
cellTsFinalNotExpire++;
@@ -631,7 +631,7 @@ public void checkProcessSummationMoreCellsSumFinalVariedTags()
tags.add(t);
byte[] tagByteArray = Tag.fromList(tags);
// create a cell with attribute SUM
- c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+ c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily, aQualifier,
cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray);
currentColumnCells.add(c1);
cellTsNotFinal++;
@@ -688,7 +688,7 @@ public void testProcessSummationMoreCellsSumFinal() throws IOException {
SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR);
// create a cell with a VERY old timestamp and attribute SUM_FINAL
- Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily,
+ Cell c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily,
aQualifier, 120L, Bytes.toBytes(cellValue1), tagByteArray);
currentColumnCells.add(c1);
@@ -699,7 +699,7 @@ public void testProcessSummationMoreCellsSumFinal() throws IOException {
tagByteArray = Tag.fromList(tags);
// create a cell with a VERY old timestamp but has attribute SUM
- Cell c2 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily,
+ Cell c2 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily,
aQualifier, 130L, Bytes.toBytes(cellValue2), tagByteArray);
currentColumnCells.add(c2);
List cells = fs.processSummationMajorCompaction(currentColumnCells,
@@ -746,7 +746,7 @@ public void testProcessSummationOneCellSumFinal() throws IOException {
SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR);
// create a cell with a VERY old timestamp
- Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily,
+ Cell c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily,
aQualifier, 120L, Bytes.toBytes(1110L), tagByteArray);
currentColumnCells.add(c1);
@@ -784,7 +784,7 @@ public void testProcessSummationOneCell() throws IOException {
SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR);
- Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily,
+ Cell c1 = HBaseTimelineSchemaUtils.createNewCell(aRowKey, aFamily,
aQualifier, currentTimestamp, Bytes.toBytes(1110L), tagByteArray);
currentColumnCells.add(c1);
List cells = fs.processSummationMajorCompaction(currentColumnCells,
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/pom.xml
new file mode 100644
index 0000000..9f919f1
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/pom.xml
@@ -0,0 +1,154 @@
+
+
+
+
+ hadoop-yarn-server-timelineservice-hbase
+ org.apache.hadoop
+ 3.0.0-alpha2-SNAPSHOT
+
+ 4.0.0
+
+ hadoop-yarn-server-timelineservice-hbase-client
+
+
+
+ commons-logging
+ commons-logging
+
+
+
+ commons-lang
+ commons-lang
+
+
+
+ commons-cli
+ commons-cli
+
+
+
+ com.google.guava
+ guava
+
+
+
+ org.apache.hadoop
+ hadoop-annotations
+
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-api
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-common
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-common
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-timelineservice
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-timelineservice-hbase-schema
+
+
+
+ org.apache.hbase
+ hbase-common
+
+
+ hadoop-common
+ org.apache.hadoop
+
+
+ org.apache.hadoop
+ hadoop-mapreduce-client-core
+
+
+ org.mortbay.jetty
+ jetty-util
+
+
+
+
+
+ org.apache.hbase
+ hbase-client
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+ org.apache.hadoop
+ hadoop-auth
+
+
+ org.apache.hadoop
+ hadoop-mapreduce-client-core
+
+
+
+
+
+
+
+
+ maven-jar-plugin
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+
+ org.apache.maven.plugins
+ maven-javadoc-plugin
+
+
+
+ junit
+ junit
+ 4.11
+
+
+
+
+
+
+
+
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
new file mode 100644
index 0000000..38ed31a
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
@@ -0,0 +1,568 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.service.AbstractService;
+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;
+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;
+import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+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.application.ApplicationTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumnPrefix;
+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.apptoflow.AppToFlowTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefixRWHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnRWHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
+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.TypedBufferedMutator;
+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.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTableRW;
+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;
+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.FlowActivityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
+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.FlowRunTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTableRW;
+
+/**
+ * This implements a hbase based backend for storing the timeline entity
+ * information.
+ * It writes to multiple tables at the backend
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HBaseTimelineWriterImpl extends AbstractService implements
+ TimelineWriter {
+
+ private static final Log LOG = LogFactory
+ .getLog(HBaseTimelineWriterImpl.class);
+
+ private Connection conn;
+ private TypedBufferedMutator entityTable;
+ private TypedBufferedMutator appToFlowTable;
+ private TypedBufferedMutator applicationTable;
+ private TypedBufferedMutator flowActivityTable;
+ private TypedBufferedMutator flowRunTable;
+
+ /**
+ * Used to convert strings key components to and from storage format.
+ */
+ private final KeyConverter stringKeyConverter =
+ new StringKeyConverter();
+
+ /**
+ * Used to convert Long key components to and from storage format.
+ */
+ private final KeyConverter longKeyConverter = new LongKeyConverter();
+
+ public HBaseTimelineWriterImpl() {
+ super(HBaseTimelineWriterImpl.class.getName());
+ }
+
+ public HBaseTimelineWriterImpl(Configuration conf) throws IOException {
+ super(conf.get("yarn.application.id",
+ HBaseTimelineWriterImpl.class.getName()));
+ }
+
+ /**
+ * initializes the hbase connection to write to the entity table.
+ */
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
+ Configuration hbaseConf =
+ HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(conf);
+ conn = ConnectionFactory.createConnection(hbaseConf);
+ entityTable = new EntityTableRW().getTableMutator(hbaseConf, conn);
+ appToFlowTable = new AppToFlowTableRW().getTableMutator(hbaseConf, conn);
+ applicationTable = new ApplicationTableRW().getTableMutator(hbaseConf, conn);
+ flowRunTable = new FlowRunTableRW().getTableMutator(hbaseConf, conn);
+ flowActivityTable =
+ new FlowActivityTableRW().getTableMutator(hbaseConf, conn);
+ }
+
+ /**
+ * Stores the entire information in TimelineEntities to the timeline store.
+ */
+ @Override
+ public TimelineWriteResponse write(String clusterId, String userId,
+ String flowName, String flowVersion, long flowRunId, String appId,
+ TimelineEntities data) throws IOException {
+
+ TimelineWriteResponse putStatus = new TimelineWriteResponse();
+ // defensive coding to avoid NPE during row key construction
+ if ((flowName == null) || (appId == null) || (clusterId == null)
+ || (userId == null)) {
+ LOG.warn("Found null for one of: flowName=" + flowName + " appId=" + appId
+ + " userId=" + userId + " clusterId=" + clusterId
+ + " . Not proceeding with writing to hbase");
+ return putStatus;
+ }
+
+ for (TimelineEntity te : data.getEntities()) {
+
+ // a set can have at most 1 null
+ if (te == null) {
+ continue;
+ }
+
+ // if the entity is the application, the destination is the application
+ // table
+ boolean isApplication = ApplicationEntity.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.getIdPrefix(), te.getId());
+ rowKey = entityRowKey.getRowKey();
+ }
+
+ storeInfo(rowKey, te, flowVersion, isApplication);
+ storeEvents(rowKey, te.getEvents(), isApplication);
+ storeConfig(rowKey, te.getConfigs(), isApplication);
+ storeMetrics(rowKey, te.getMetrics(), isApplication);
+ storeRelations(rowKey, te, isApplication);
+
+ if (isApplication) {
+ TimelineEvent event =
+ ApplicationEntity.getApplicationEvent(te,
+ ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+ FlowRunRowKey flowRunRowKey =
+ new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
+ if (event != null) {
+ onApplicationCreated(flowRunRowKey, clusterId, appId, userId,
+ flowVersion, te, event.getTimestamp());
+ }
+ // if it's an application entity, store metrics
+ storeFlowMetricsAppRunning(flowRunRowKey, appId, te);
+ // if application has finished, store it's finish time and write final
+ // values of all metrics
+ event = ApplicationEntity.getApplicationEvent(te,
+ ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+ if (event != null) {
+ onApplicationFinished(flowRunRowKey, flowVersion, appId, te,
+ event.getTimestamp());
+ }
+ }
+ }
+ return putStatus;
+ }
+
+ private void onApplicationCreated(FlowRunRowKey flowRunRowKey,
+ String clusterId, String appId, String userId, String flowVersion,
+ TimelineEntity te, long appCreatedTimeStamp)
+ throws IOException {
+
+ String flowName = flowRunRowKey.getFlowName();
+ Long flowRunId = flowRunRowKey.getFlowRunId();
+
+ // store in App to flow table
+ AppToFlowRowKey appToFlowRowKey = new AppToFlowRowKey(appId);
+ byte[] rowKey = appToFlowRowKey.getRowKey();
+ ColumnPrefixRWHelper.store(rowKey, appToFlowTable,
+ AppToFlowColumnPrefix.FLOW_NAME, clusterId, null, flowName);
+ ColumnPrefixRWHelper.store(rowKey, appToFlowTable,
+ AppToFlowColumnPrefix.FLOW_RUN_ID, clusterId, null, flowRunId);
+ ColumnPrefixRWHelper.store(rowKey, appToFlowTable,
+ AppToFlowColumnPrefix.USER_ID, clusterId, 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());
+ ColumnPrefixRWHelper.store(flowActivityRowKeyBytes, flowActivityTable,
+ FlowActivityColumnPrefix.RUN_ID, qualifier, null, flowVersion,
+ AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
+ }
+
+ /*
+ * updates the {@link FlowRunTableRW} with Application Created information
+ */
+ private void storeAppCreatedInFlowRunTable(FlowRunRowKey flowRunRowKey,
+ String appId, TimelineEntity te) throws IOException {
+ byte[] rowKey = flowRunRowKey.getRowKey();
+ ColumnRWHelper.store(rowKey, flowRunTable, FlowRunColumn.MIN_START_TIME,
+ null, te.getCreatedTime(),
+ AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
+ }
+
+
+ /*
+ * updates the {@link FlowRunTableRW} and {@link FlowActivityTableRW} when an
+ * application has finished
+ */
+ private void onApplicationFinished(FlowRunRowKey flowRunRowKey,
+ String flowVersion, String appId, TimelineEntity te,
+ long appFinishedTimeStamp) throws IOException {
+ // store in flow run table
+ storeAppFinishedInFlowRunTable(flowRunRowKey, appId, te,
+ appFinishedTimeStamp);
+
+ // indicate in the flow activity table that the app has finished
+ byte[] rowKey =
+ new FlowActivityRowKey(flowRunRowKey.getClusterId(),
+ appFinishedTimeStamp, flowRunRowKey.getUserId(),
+ flowRunRowKey.getFlowName()).getRowKey();
+ byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId());
+ ColumnPrefixRWHelper.store(rowKey, flowActivityTable,
+ FlowActivityColumnPrefix.RUN_ID, qualifier, null, flowVersion,
+ AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
+ }
+
+ /*
+ * Update the {@link FlowRunTableRW} with Application Finished information
+ */
+ private void storeAppFinishedInFlowRunTable(FlowRunRowKey flowRunRowKey,
+ String appId, TimelineEntity te, long appFinishedTimeStamp)
+ throws IOException {
+ byte[] rowKey = flowRunRowKey.getRowKey();
+ Attribute attributeAppId =
+ AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId);
+ ColumnRWHelper.store(rowKey, flowRunTable, FlowRunColumn.MAX_END_TIME,
+ null, appFinishedTimeStamp, attributeAppId);
+
+ // store the final value of metrics since application has finished
+ Set metrics = te.getMetrics();
+ if (metrics != null) {
+ storeFlowMetrics(rowKey, metrics, attributeAppId,
+ AggregationOperation.SUM_FINAL.getAttribute());
+ }
+ }
+
+ /*
+ * Updates the {@link FlowRunTableRW} with Application Metrics
+ */
+ private void storeFlowMetricsAppRunning(FlowRunRowKey flowRunRowKey,
+ String appId, TimelineEntity te) throws IOException {
+ Set metrics = te.getMetrics();
+ if (metrics != null) {
+ byte[] rowKey = flowRunRowKey.getRowKey();
+ storeFlowMetrics(rowKey, metrics,
+ AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId),
+ AggregationOperation.SUM.getAttribute());
+ }
+ }
+
+ private void storeFlowMetrics(byte[] rowKey, Set metrics,
+ Attribute... attributes) throws IOException {
+ for (TimelineMetric metric : metrics) {
+ byte[] metricColumnQualifier = stringKeyConverter.encode(metric.getId());
+ Map timeseries = metric.getValues();
+ for (Map.Entry timeseriesEntry : timeseries.entrySet()) {
+ Long timestamp = timeseriesEntry.getKey();
+ ColumnPrefixRWHelper.store(rowKey, flowRunTable,
+ FlowRunColumnPrefix.METRIC, metricColumnQualifier,
+ timestamp, timeseriesEntry.getValue(), attributes);
+ }
+ }
+ }
+
+ private void storeRelations(byte[] rowKey, TimelineEntity te,
+ boolean isApplication) throws IOException {
+ if (isApplication) {
+ storeRelations(rowKey, te.getIsRelatedToEntities(),
+ ApplicationColumnPrefix.IS_RELATED_TO, applicationTable);
+ storeRelations(rowKey, te.getRelatesToEntities(),
+ ApplicationColumnPrefix.RELATES_TO, applicationTable);
+ } else {
+ storeRelations(rowKey, te.getIsRelatedToEntities(),
+ EntityColumnPrefix.IS_RELATED_TO, entityTable);
+ storeRelations(rowKey, te.getRelatesToEntities(),
+ EntityColumnPrefix.RELATES_TO, entityTable);
+ }
+ }
+
+ /**
+ * Stores the Relations from the {@linkplain TimelineEntity} object.
+ */
+ private void storeRelations(byte[] rowKey,
+ Map> connectedEntities,
+ ColumnPrefix columnPrefix, TypedBufferedMutator table)
+ throws IOException {
+ for (Map.Entry> connectedEntity : connectedEntities
+ .entrySet()) {
+ // id3?id4?id5
+ String compoundValue =
+ Separator.VALUES.joinEncoded(connectedEntity.getValue());
+ ColumnPrefixRWHelper.store(rowKey, table, columnPrefix,
+ stringKeyConverter.encode(connectedEntity.getKey()),
+ null, compoundValue);
+ }
+ }
+
+ /**
+ * Stores information from the {@linkplain TimelineEntity} object.
+ */
+ private void storeInfo(byte[] rowKey, TimelineEntity te, String flowVersion,
+ boolean isApplication) throws IOException {
+
+ if (isApplication) {
+ ColumnRWHelper.store(rowKey, applicationTable, ApplicationColumn.ID,
+ null, te.getId());
+ ColumnRWHelper.store(rowKey, applicationTable,
+ ApplicationColumn.CREATED_TIME, null, te.getCreatedTime());
+ ColumnRWHelper.store(rowKey, applicationTable,
+ ApplicationColumn.FLOW_VERSION, null, flowVersion);
+ Map info = te.getInfo();
+ if (info != null) {
+ for (Map.Entry entry : info.entrySet()) {
+ ColumnPrefixRWHelper.store(rowKey, applicationTable,
+ ApplicationColumnPrefix.INFO,
+ stringKeyConverter.encode(entry.getKey()),
+ null, entry.getValue());
+ }
+ }
+ } else {
+ ColumnRWHelper.store(rowKey, entityTable, EntityColumn.ID, null,
+ te.getId());
+ ColumnRWHelper.store(rowKey, entityTable, EntityColumn.TYPE, null,
+ te.getType());
+ ColumnRWHelper.store(rowKey, entityTable, EntityColumn.CREATED_TIME,
+ null, te.getCreatedTime());
+ ColumnRWHelper.store(rowKey, entityTable, EntityColumn.FLOW_VERSION,
+ null, flowVersion);
+ Map info = te.getInfo();
+ if (info != null) {
+ for (Map.Entry entry : info.entrySet()) {
+ ColumnPrefixRWHelper.store(rowKey, entityTable,
+ EntityColumnPrefix.INFO,
+ stringKeyConverter.encode(entry.getKey()),
+ null, entry.getValue());
+ }
+ }
+ }
+ }
+
+ /**
+ * stores the config information from {@linkplain TimelineEntity}.
+ */
+ private void storeConfig(byte[] rowKey, Map config,
+ boolean isApplication) throws IOException {
+ if (config == null) {
+ return;
+ }
+ for (Map.Entry entry : config.entrySet()) {
+ byte[] configKey = stringKeyConverter.encode(entry.getKey());
+ if (isApplication) {
+ ColumnPrefixRWHelper.store(rowKey, applicationTable,
+ ApplicationColumnPrefix.CONFIG,
+ configKey, null, entry.getValue());
+ } else {
+ ColumnPrefixRWHelper.store(rowKey, entityTable,
+ ApplicationColumnPrefix.CONFIG, configKey,
+ null, entry.getValue());
+ }
+ }
+ }
+
+ /**
+ * stores the {@linkplain TimelineMetric} information from the
+ * {@linkplain TimelineEvent} object.
+ */
+ private void storeMetrics(byte[] rowKey, Set metrics,
+ boolean isApplication) throws IOException {
+ if (metrics != null) {
+ for (TimelineMetric metric : metrics) {
+ byte[] metricColumnQualifier =
+ stringKeyConverter.encode(metric.getId());
+ Map timeseries = metric.getValues();
+ for (Map.Entry timeseriesEntry : timeseries.entrySet()) {
+ Long timestamp = timeseriesEntry.getKey();
+ if (isApplication) {
+ ColumnPrefixRWHelper.store(rowKey, applicationTable,
+ ApplicationColumnPrefix.METRIC,
+ metricColumnQualifier, timestamp, timeseriesEntry.getValue());
+ } else {
+ ColumnPrefixRWHelper.store(rowKey, entityTable,
+ EntityColumnPrefix.METRIC,
+ metricColumnQualifier, timestamp, timeseriesEntry.getValue());
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Stores the events from the {@linkplain TimelineEvent} object.
+ */
+ private void storeEvents(byte[] rowKey, Set events,
+ boolean isApplication) throws IOException {
+ if (events != null) {
+ for (TimelineEvent event : events) {
+ if (event != null) {
+ String eventId = event.getId();
+ if (eventId != null) {
+ long eventTimestamp = event.getTimestamp();
+ // if the timestamp is not set, use the current timestamp
+ if (eventTimestamp == TimelineEvent.INVALID_TIMESTAMP) {
+ LOG.warn("timestamp is not set for event " + eventId +
+ "! Using the current timestamp");
+ eventTimestamp = System.currentTimeMillis();
+ }
+ Map eventInfo = event.getInfo();
+ if ((eventInfo == null) || (eventInfo.size() == 0)) {
+ byte[] columnQualifierBytes =
+ new EventColumnName(eventId, eventTimestamp, null)
+ .getColumnQualifier();
+ if (isApplication) {
+ ColumnPrefixRWHelper.store(rowKey, applicationTable,
+ ApplicationColumnPrefix.EVENT,
+ columnQualifierBytes, null, Separator.EMPTY_BYTES);
+ } else {
+ ColumnPrefixRWHelper.store(rowKey, entityTable,
+ EntityColumnPrefix.EVENT,
+ columnQualifierBytes, null, Separator.EMPTY_BYTES);
+ }
+ } else {
+ for (Map.Entry info : eventInfo.entrySet()) {
+ // eventId=infoKey
+ byte[] columnQualifierBytes =
+ new EventColumnName(eventId, eventTimestamp, info.getKey())
+ .getColumnQualifier();
+ if (isApplication) {
+ ColumnPrefixRWHelper.store(rowKey, applicationTable,
+ ApplicationColumnPrefix.EVENT,
+ columnQualifierBytes, null, info.getValue());
+ } else {
+ ColumnPrefixRWHelper.store(rowKey, entityTable,
+ ApplicationColumnPrefix.EVENT,
+ columnQualifierBytes, null, info.getValue());
+ }
+ } // for info: eventInfo
+ }
+ }
+ }
+ } // event : events
+ }
+ }
+
+ /*
+ * (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
+ entityTable.flush();
+ appToFlowTable.flush();
+ applicationTable.flush();
+ flowRunTable.flush();
+ flowActivityTable.flush();
+ }
+
+ /**
+ * close the hbase connections The close APIs perform flushing and release any
+ * resources held.
+ */
+ @Override
+ protected void serviceStop() throws Exception {
+ if (entityTable != null) {
+ LOG.info("closing the entity table");
+ // The close API performs flushing and releases any resources held
+ entityTable.close();
+ }
+ if (appToFlowTable != null) {
+ LOG.info("closing the app_flow table");
+ // The close API performs flushing and releases any resources held
+ appToFlowTable.close();
+ }
+ if (applicationTable != null) {
+ LOG.info("closing the application table");
+ applicationTable.close();
+ }
+ if (flowRunTable != null) {
+ LOG.info("closing the flow run table");
+ // The close API performs flushing and releases any resources held
+ flowRunTable.close();
+ }
+ if (flowActivityTable != null) {
+ LOG.info("closing the flowActivityTable table");
+ // The close API performs flushing and releases any resources held
+ flowActivityTable.close();
+ }
+ if (conn != null) {
+ LOG.info("closing the hbase Connection");
+ conn.close();
+ }
+ super.serviceStop();
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
similarity index 92%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index dd87169..ea873df 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -38,12 +38,12 @@
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.util.GenericOptionsParser;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTableRW;
import com.google.common.annotations.VisibleForTesting;
@@ -80,24 +80,24 @@ public static void main(String[] args) throws Exception {
String entityTableName
= commandLine.getOptionValue(ENTITY_TABLE_NAME_SHORT);
if (StringUtils.isNotBlank(entityTableName)) {
- hbaseConf.set(EntityTable.TABLE_NAME_CONF_NAME, entityTableName);
+ hbaseConf.set(EntityTableRW.TABLE_NAME_CONF_NAME, entityTableName);
}
String entityTableTTLMetrics = commandLine.getOptionValue(TTL_OPTION_SHORT);
if (StringUtils.isNotBlank(entityTableTTLMetrics)) {
int metricsTTL = Integer.parseInt(entityTableTTLMetrics);
- new EntityTable().setMetricsTTL(metricsTTL, hbaseConf);
+ new EntityTableRW().setMetricsTTL(metricsTTL, hbaseConf);
}
// Grab the appToflowTableName argument
String appToflowTableName = commandLine.getOptionValue(
APP_TO_FLOW_TABLE_NAME_SHORT);
if (StringUtils.isNotBlank(appToflowTableName)) {
- hbaseConf.set(AppToFlowTable.TABLE_NAME_CONF_NAME, appToflowTableName);
+ hbaseConf.set(AppToFlowTableRW.TABLE_NAME_CONF_NAME, appToflowTableName);
}
// Grab the applicationTableName argument
String applicationTableName = commandLine.getOptionValue(
APP_TABLE_NAME_SHORT);
if (StringUtils.isNotBlank(applicationTableName)) {
- hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME,
+ hbaseConf.set(ApplicationTableRW.TABLE_NAME_CONF_NAME,
applicationTableName);
}
@@ -196,7 +196,7 @@ public static void createAllTables(Configuration hbaseConf,
throw new IOException("Cannot create table since admin is null");
}
try {
- new EntityTable().createTable(admin, hbaseConf);
+ new EntityTableRW().createTable(admin, hbaseConf);
} catch (IOException e) {
if (skipExisting) {
LOG.warn("Skip and continue on: " + e.getMessage());
@@ -205,7 +205,7 @@ public static void createAllTables(Configuration hbaseConf,
}
}
try {
- new AppToFlowTable().createTable(admin, hbaseConf);
+ new AppToFlowTableRW().createTable(admin, hbaseConf);
} catch (IOException e) {
if (skipExisting) {
LOG.warn("Skip and continue on: " + e.getMessage());
@@ -214,7 +214,7 @@ public static void createAllTables(Configuration hbaseConf,
}
}
try {
- new ApplicationTable().createTable(admin, hbaseConf);
+ new ApplicationTableRW().createTable(admin, hbaseConf);
} catch (IOException e) {
if (skipExisting) {
LOG.warn("Skip and continue on: " + e.getMessage());
@@ -223,7 +223,7 @@ public static void createAllTables(Configuration hbaseConf,
}
}
try {
- new FlowRunTable().createTable(admin, hbaseConf);
+ new FlowRunTableRW().createTable(admin, hbaseConf);
} catch (IOException e) {
if (skipExisting) {
LOG.warn("Skip and continue on: " + e.getMessage());
@@ -232,7 +232,7 @@ public static void createAllTables(Configuration hbaseConf,
}
}
try {
- new FlowActivityTable().createTable(admin, hbaseConf);
+ new FlowActivityTableRW().createTable(admin, hbaseConf);
} catch (IOException e) {
if (skipExisting) {
LOG.warn("Skip and continue on: " + e.getMessage());
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTableRW.java
similarity index 97%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTableRW.java
index a02f768..f4bbc7a 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTableRW.java
@@ -28,7 +28,7 @@
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
/**
@@ -67,7 +67,7 @@
* |-------------------------------------------------------------------------|
*
*/
-public class ApplicationTable extends BaseTable {
+public class ApplicationTableRW extends BaseTableRW {
/** application prefix. */
private static final String PREFIX =
YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".application";
@@ -92,9 +92,9 @@
/** default max number of versions. */
private static final int DEFAULT_METRICS_MAX_VERSIONS = 1000;
- private static final Log LOG = LogFactory.getLog(ApplicationTable.class);
+ private static final Log LOG = LogFactory.getLog(ApplicationTableRW.class);
- public ApplicationTable() {
+ public ApplicationTableRW() {
super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);
}
@@ -102,7 +102,7 @@ public ApplicationTable() {
* (non-Javadoc)
*
* @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable
+ * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW#createTable
* (org.apache.hadoop.hbase.client.Admin,
* org.apache.hadoop.conf.Configuration)
*/
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTableRW.java
similarity index 97%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTableRW.java
index 583ee04..54786fd 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTableRW.java
@@ -27,7 +27,7 @@
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
import java.io.IOException;
@@ -68,7 +68,7 @@
* may be more than one applications for a given app id. Different clusters are
* recorded as different sets of columns.
*/
-public class AppToFlowTable extends BaseTable {
+public class AppToFlowTableRW extends BaseTableRW {
/** app_flow prefix. */
private static final String PREFIX =
YarnConfiguration.TIMELINE_SERVICE_PREFIX + "app-flow";
@@ -79,9 +79,9 @@
/** default value for app_flow table name. */
private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow";
- private static final Log LOG = LogFactory.getLog(AppToFlowTable.class);
+ private static final Log LOG = LogFactory.getLog(AppToFlowTableRW.class);
- public AppToFlowTable() {
+ public AppToFlowTableRW() {
super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);
}
@@ -89,7 +89,7 @@ public AppToFlowTable() {
* (non-Javadoc)
*
* @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable
+ * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW#createTable
* (org.apache.hadoop.hbase.client.Admin,
* org.apache.hadoop.conf.Configuration)
*/
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTableRW.java
similarity index 98%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTableRW.java
index 93d809c..4f637a3 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTableRW.java
@@ -37,7 +37,7 @@
*
* @param reference to the table instance class itself for type safety.
*/
-public abstract class BaseTable {
+public abstract class BaseTableRW {
/**
* Name of config variable that is used to point to this table.
@@ -56,7 +56,7 @@
* @param defaultTableName Default table name if table from config is not
* found.
*/
- protected BaseTable(String tableNameConfName, String defaultTableName) {
+ protected BaseTableRW(String tableNameConfName, String defaultTableName) {
this.tableNameConfName = tableNameConfName;
this.defaultTableName = defaultTableName;
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefixRWHelper.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefixRWHelper.java
new file mode 100644
index 0000000..0bb4f76
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefixRWHelper.java
@@ -0,0 +1,166 @@
+/**
+ * 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 org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.NavigableMap;
+
+public final class ColumnPrefixRWHelper {
+ private ColumnPrefixRWHelper() {
+ }
+
+ /**
+ * Sends a Mutation to the table. The mutations will be buffered and sent over
+ * the wire as part of a batch.
+ *
+ * @param rowKey identifying the row to write. Nothing gets written when null.
+ * @param tableMutator used to modify the underlying HBase table. Caller is
+ * responsible to pass a mutator for the table that actually has this
+ * column.
+ * @param qualifier column qualifier. Nothing gets written when null.
+ * @param timestamp version timestamp. When null the server timestamp will be
+ * used.
+ * @param attributes attributes for the mutation that are used by the
+ * coprocessor to set/read the cell tags.
+ * @param inputValue the value to write to the rowKey and column qualifier.
+ * Nothing gets written when null.
+ * @throws IOException if there is any exception encountered while doing
+ * store operation(sending mutation to the table).
+ */
+ public static void store(byte[] rowKey, TypedBufferedMutator> tableMutator,
+ ColumnPrefix> columnPrefix, byte[] qualifier, Long timestamp,
+ Object inputValue, Attribute... attributes) throws IOException {
+ // Null check
+ if (qualifier == null) {
+ throw new IOException("Cannot store column with null qualifier in "
+ +tableMutator.getName().getNameAsString());
+ }
+
+ byte[] columnQualifier = columnPrefix.getColumnPrefixBytes(qualifier);
+ Attribute[] combinedAttributes =
+ columnPrefix.combineAttrsWithAggr(attributes);
+
+ ColumnRWHelper.store(rowKey, tableMutator,
+ columnPrefix.getColumnFamilyBytes(), columnQualifier, timestamp,
+ inputValue, columnPrefix.getValueConverter(),combinedAttributes);
+ }
+
+ /**
+ * Sends a Mutation to the table. The mutations will be buffered and sent over
+ * the wire as part of a batch.
+ *
+ * @param rowKey identifying the row to write. Nothing gets written when null.
+ * @param tableMutator used to modify the underlying HBase table. Caller is
+ * responsible to pass a mutator for the table that actually has this
+ * column.
+ * @param qualifier column qualifier. Nothing gets written when null.
+ * @param timestamp version timestamp. When null the server timestamp will be
+ * used.
+ * @param attributes attributes for the mutation that are used by the
+ * coprocessor to set/read the cell tags.
+ * @param inputValue the value to write to the rowKey and column qualifier.
+ * Nothing gets written when null.
+ * @throws IOException if there is any exception encountered while doing
+ * store operation(sending mutation to the table).
+ */
+ public static void store(byte[] rowKey, TypedBufferedMutator> tableMutator,
+ ColumnPrefix> columnPrefix, String qualifier, Long timestamp,
+ Object inputValue, Attribute... attributes) throws IOException {
+ // Null check
+ if (qualifier == null) {
+ throw new IOException("Cannot store column with null qualifier in "
+ + tableMutator.getName().getNameAsString());
+ }
+
+ byte[] columnQualifier = columnPrefix.getColumnPrefixBytes(qualifier);
+ Attribute[] combinedAttributes =
+ columnPrefix.combineAttrsWithAggr(attributes);
+
+ ColumnRWHelper.store(
+ rowKey, tableMutator, columnPrefix.getColumnFamilyBytes(),
+ columnQualifier, timestamp, inputValue,
+ columnPrefix.getValueConverter(), combinedAttributes);
+ }
+
+ /**
+ * Get the latest version of this specified column. Note: this call clones the
+ * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
+ *
+ * @param result Cannot be null
+ * @param qualifier column qualifier. Nothing gets read when null.
+ * @return result object (can be cast to whatever object was written to) or
+ * null when specified column qualifier for this prefix doesn't exist
+ * in the result.
+ * @throws IOException if there is any exception encountered while reading
+ * result.
+ */
+ public static Object readResult(Result result, ColumnPrefix> columnPrefix,
+ String qualifier) throws IOException {
+ byte[] columnQualifier = ColumnHelper.getColumnQualifier(
+ columnPrefix.getColumnPrefixBytes(), qualifier);
+
+ return ColumnRWHelper.readResult(
+ result, columnPrefix.getColumnFamilyBytes(),
+ columnQualifier, columnPrefix.getValueConverter());
+ }
+
+ /**
+ *
+ * @param identifies the type of key converter.
+ * @param result from which to read columns.
+ * @param keyConverter used to convert column bytes to the appropriate key
+ * type
+ * @return the latest values of columns in the column family with this prefix
+ * (or all of them if the prefix value is null).
+ * @throws IOException if there is any exception encountered while reading
+ * results.
+ */
+ public static Map readResults(Result result,
+ ColumnPrefix> columnPrefix, KeyConverter keyConverter)
+ throws IOException {
+ return ColumnRWHelper.readResults(result,
+ columnPrefix.getColumnFamilyBytes(),
+ columnPrefix.getColumnPrefixBytes(),
+ keyConverter, columnPrefix.getValueConverter());
+ }
+
+ /**
+ * @param result from which to reads data with timestamps.
+ * @param identifies the type of key converter.
+ * @param the type of the values. The values will be cast into that type.
+ * @param keyConverter used to convert column bytes to the appropriate key
+ * type.
+ * @return the cell values at each respective time in for form
+ * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2},
+ * idB={timestamp3->value3}, idC={timestamp1->value4}}}
+ * @throws IOException if there is any exception encountered while reading
+ * result.
+ */
+ public static NavigableMap>
+ readResultsWithTimestamps(Result result, ColumnPrefix> columnPrefix,
+ KeyConverter keyConverter) throws IOException {
+ return ColumnRWHelper.readResultsWithTimestamps(result,
+ columnPrefix.getColumnFamilyBytes(),
+ columnPrefix.getColumnPrefixBytes(),
+ keyConverter, columnPrefix.getValueConverter());
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnRWHelper.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnRWHelper.java
new file mode 100644
index 0000000..07247d4
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnRWHelper.java
@@ -0,0 +1,341 @@
+/**
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+/**
+ * A utility class to read/modify a column.
+ */
+public final class ColumnRWHelper {
+ private static final Log LOG = LogFactory.getLog(ColumnRWHelper.class);
+
+ private ColumnRWHelper() {
+ }
+
+ /**
+ * Sends a Mutation to the table. The mutations will be buffered and sent over
+ * the wire as part of a batch.
+ *
+ * @param rowKey
+ * identifying the row to write. Nothing gets written when null.
+ * @param tableMutator
+ * used to modify the underlying HBase table
+ * @param column the column that is to be modified
+ * @param timestamp
+ * version timestamp. When null the current timestamp multiplied with
+ * TimestampGenerator.TS_MULTIPLIER and added with last 3 digits of
+ * app id will be used
+ * @param inputValue
+ * the value to write to the rowKey and column qualifier. Nothing
+ * gets written when null.
+ * @param attributes Attributes to be set for HBase Put.
+ * @throws IOException if any problem occurs during store operation(sending
+ * mutation to table).
+ */
+ public static void store(byte[] rowKey, TypedBufferedMutator> tableMutator,
+ Column> column, Long timestamp,
+ Object inputValue, Attribute... attributes)
+ throws IOException {
+ store(rowKey, tableMutator, column.getColumnFamilyBytes(),
+ column.getColumnQualifierBytes(), timestamp, inputValue,
+ column.getValueConverter(), column.combineAttrsWithAggr(attributes));
+ }
+
+ /**
+ * Sends a Mutation to the table. The mutations will be buffered and sent over
+ * the wire as part of a batch.
+ *
+ * @param rowKey
+ * identifying the row to write. Nothing gets written when null.
+ * @param tableMutator
+ * used to modify the underlying HBase table
+ * @param columnFamilyBytes
+ * @param columnQualifier
+ * column qualifier. Nothing gets written when null.
+ * @param timestamp
+ * version timestamp. When null the current timestamp multiplied with
+ * TimestampGenerator.TS_MULTIPLIER and added with last 3 digits of
+ * app id will be used
+ * @param inputValue
+ * the value to write to the rowKey and column qualifier. Nothing
+ * gets written when null.
+ * @param converter
+ * @param attributes Attributes to be set for HBase Put.
+ * @throws IOException if any problem occurs during store operation(sending
+ * mutation to table).
+ */
+ public static void store(byte[] rowKey, TypedBufferedMutator> tableMutator,
+ byte[] columnFamilyBytes, byte[] columnQualifier, Long timestamp,
+ Object inputValue, ValueConverter converter, Attribute... attributes)
+ throws IOException {
+ if ((rowKey == null) || (columnQualifier == null) || (inputValue == null)) {
+ return;
+ }
+ Put p = new Put(rowKey);
+ timestamp = getPutTimestamp(timestamp, attributes);
+ p.addColumn(columnFamilyBytes, columnQualifier, timestamp,
+ converter.encodeValue(inputValue));
+ if ((attributes != null) && (attributes.length > 0)) {
+ for (Attribute attribute : attributes) {
+ p.setAttribute(attribute.getName(), attribute.getValue());
+ }
+ }
+ tableMutator.mutate(p);
+ }
+
+ /**
+ * Figures out the cell timestamp used in the Put For storing into flow run
+ * table. We would like to left shift the timestamp and supplement it with the
+ * AppId id so that there are no collisions in the flow run table's cells
+ */
+ private static long getPutTimestamp(Long timestamp, Attribute[] attributes) {
+ if (timestamp == null) {
+ timestamp = System.currentTimeMillis();
+ }
+ String appId = getAppIdFromAttributes(attributes);
+ long supplementedTS = TimestampGenerator.getSupplementedTimestamp(
+ timestamp, appId);
+ return supplementedTS;
+ }
+
+ private static String getAppIdFromAttributes(Attribute[] attributes) {
+ if (attributes == null) {
+ return null;
+ }
+ String appId = null;
+ for (Attribute attribute : attributes) {
+ if (AggregationCompactionDimension.APPLICATION_ID.toString().equals(
+ attribute.getName())) {
+ appId = Bytes.toString(attribute.getValue());
+ }
+ }
+ return appId;
+ }
+
+ /**
+ * Get the latest version of this specified column. Note: this call clones the
+ * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
+ *
+ * @param result from which to read the value. Cannot be null
+ * @param columnFamilyBytes
+ * @param columnQualifierBytes referring to the column to be read.
+ * @param converter
+ * @return latest version of the specified column of whichever object was
+ * written.
+ * @throws IOException if any problem occurs while reading result.
+ */
+ public static Object readResult(Result result, byte[] columnFamilyBytes,
+ byte[] columnQualifierBytes, ValueConverter converter)
+ throws IOException {
+ if (result == null || columnQualifierBytes == null) {
+ return null;
+ }
+
+ // Would have preferred to be able to use getValueAsByteBuffer and get a
+ // ByteBuffer to avoid copy, but GenericObjectMapper doesn't seem to like
+ // that.
+ byte[] value = result.getValue(columnFamilyBytes, columnQualifierBytes);
+ return converter.decodeValue(value);
+ }
+
+ /**
+ * Get the latest version of this specified column. Note: this call clones the
+ * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
+ *
+ * @param result from which to read the value. Cannot be null
+ * @param column the column that the result can be parsed to
+ * @return latest version of the specified column of whichever object was
+ * written.
+ * @throws IOException if any problem occurs while reading result.
+ */
+ public static Object readResult(Result result, Column> column)
+ throws IOException {
+ return readResult(result, column.getColumnFamilyBytes(),
+ column.getColumnQualifierBytes(), column.getValueConverter());
+ }
+
+ /**
+ * @param identifies the type of column name(indicated by type of key
+ * converter).
+ * @param the type of the values. The values will be cast into that type.
+ * @param result from which to reads data with timestamps
+ * @param columnFamilyBytes
+ * @param columnPrefixBytes optional prefix to limit columns. If null all
+ * columns are returned.
+ * @param keyConverter used to convert column bytes to the appropriate key
+ * type.
+ * @param converter
+ * @return the cell values at each respective time in for form
+ * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2},
+ * idB={timestamp3->value3}, idC={timestamp1->value4}}}
+ * @throws IOException if any problem occurs while reading results.
+ */
+ @SuppressWarnings("unchecked")
+ public static NavigableMap>
+ readResultsWithTimestamps(Result result, byte[] columnFamilyBytes,
+ byte[] columnPrefixBytes, KeyConverter keyConverter,
+ ValueConverter converter) throws IOException {
+
+ NavigableMap> results = new TreeMap<>();
+
+ if (result != null) {
+ NavigableMap<
+ byte[], NavigableMap>> resultMap =
+ result.getMap();
+
+ NavigableMap> columnCellMap =
+ resultMap.get(columnFamilyBytes);
+
+ // could be that there is no such column family.
+ if (columnCellMap != null) {
+ for (Map.Entry> entry : columnCellMap
+ .entrySet()) {
+ K converterColumnKey = null;
+ if (columnPrefixBytes == null) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("null prefix was specified; returning all columns");
+ }
+ try {
+ converterColumnKey = keyConverter.decode(entry.getKey());
+ } catch (IllegalArgumentException iae) {
+ LOG.error("Illegal column found, skipping this column.", iae);
+ continue;
+ }
+ } else {
+ // A non-null prefix means columns are actually of the form
+ // prefix!columnNameRemainder
+ byte[][] columnNameParts =
+ Separator.QUALIFIERS.split(entry.getKey(), 2);
+ byte[] actualColumnPrefixBytes = columnNameParts[0];
+ if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes)
+ && columnNameParts.length == 2) {
+ try {
+ // This is the prefix that we want
+ converterColumnKey = keyConverter.decode(columnNameParts[1]);
+ } catch (IllegalArgumentException iae) {
+ LOG.error("Illegal column found, skipping this column.", iae);
+ continue;
+ }
+ }
+ }
+
+ // If this column has the prefix we want
+ if (converterColumnKey != null) {
+ NavigableMap cellResults =
+ new TreeMap();
+ NavigableMap cells = entry.getValue();
+ if (cells != null) {
+ for (Map.Entry cell : cells.entrySet()) {
+ V value =
+ (V) converter.decodeValue(cell.getValue());
+ cellResults.put(
+ TimestampGenerator.getTruncatedTimestamp(cell.getKey()),
+ value);
+ }
+ }
+ results.put(converterColumnKey, cellResults);
+ }
+ } // for entry : columnCellMap
+ } // if columnCellMap != null
+ } // if result != null
+ return results;
+ }
+
+ /**
+ * @param identifies the type of column name(indicated by type of key
+ * converter).
+ * @param result from which to read columns
+ * @param columnFamilyBytes
+ * @param columnPrefixBytes optional prefix to limit columns. If null all
+ * columns are returned.
+ * @param keyConverter used to convert column bytes to the appropriate key
+ * type.
+ * @param converter
+ * @return the latest values of columns in the column family. If the column
+ * prefix is null, the column qualifier is returned as Strings. For a
+ * non-null column prefix bytes, the column qualifier is returned as
+ * a list of parts, each part a byte[]. This is to facilitate
+ * returning byte arrays of values that were not Strings.
+ * @throws IOException if any problem occurs while reading results.
+ */
+ public static Map readResults(Result result,
+ byte[] columnFamilyBytes, byte[] columnPrefixBytes,
+ KeyConverter keyConverter, ValueConverter converter)
+ throws IOException {
+ Map results = new HashMap<>();
+
+ if (result != null) {
+ Map columns = result.getFamilyMap(columnFamilyBytes);
+ for (Map.Entry entry : columns.entrySet()) {
+ byte[] columnKey = entry.getKey();
+ if (columnKey != null && columnKey.length > 0) {
+
+ K converterColumnKey = null;
+ if (columnPrefixBytes == null) {
+ try {
+ converterColumnKey = keyConverter.decode(columnKey);
+ } catch (IllegalArgumentException iae) {
+ LOG.error("Illegal column found, skipping this column.", iae);
+ continue;
+ }
+ } else {
+ // A non-null prefix means columns are actually of the form
+ // prefix!columnNameRemainder
+ byte[][] columnNameParts = Separator.QUALIFIERS.split(columnKey, 2);
+ if (columnNameParts.length > 0) {
+ byte[] actualColumnPrefixBytes = columnNameParts[0];
+ // If this is the prefix that we want
+ if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes)
+ && columnNameParts.length == 2) {
+ try {
+ converterColumnKey = keyConverter.decode(columnNameParts[1]);
+ } catch (IllegalArgumentException iae) {
+ LOG.error("Illegal column found, skipping this column.", iae);
+ continue;
+ }
+ }
+ }
+ } // if-else
+
+ // If the columnPrefix is null (we want all columns), or the actual
+ // prefix matches the given prefix we want this column
+ if (converterColumnKey != null) {
+ Object value = converter.decodeValue(entry.getValue());
+ // we return the columnQualifier in parts since we don't know
+ // which part is of which data type.
+ results.put(converterColumnKey, value);
+ }
+ }
+ } // for entry
+ }
+ return results;
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
new file mode 100644
index 0000000..899d8ba
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
@@ -0,0 +1,106 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Arrays;
+
+/**
+ * A bunch of utility functions used in HBase TimelineService backend.
+ */
+public final class HBaseTimelineStorageUtils {
+
+ private HBaseTimelineStorageUtils() {
+ }
+
+ /**
+ * @param conf Yarn configuration. Used to see if there is an explicit config
+ * pointing to the HBase config file to read. If null then a new
+ * HBase configuration will be returned.
+ * @return a configuration with the HBase configuration from the classpath,
+ * optionally overwritten by the timeline service configuration URL if
+ * specified.
+ * @throws MalformedURLException if a timeline service HBase configuration URL
+ * is specified but is a malformed URL.
+ */
+ public static Configuration getTimelineServiceHBaseConf(Configuration conf)
+ throws MalformedURLException {
+ Configuration hbaseConf;
+
+ if (conf == null) {
+ return HBaseConfiguration.create();
+ }
+
+ String timelineServiceHBaseConfFileURL =
+ conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE);
+ if (timelineServiceHBaseConfFileURL != null
+ && timelineServiceHBaseConfFileURL.length() > 0) {
+ // create a clone so that we don't mess with out input one
+ hbaseConf = new Configuration(conf);
+ Configuration plainHBaseConf = new Configuration(false);
+ URL hbaseSiteXML = new URL(timelineServiceHBaseConfFileURL);
+ plainHBaseConf.addResource(hbaseSiteXML);
+ HBaseConfiguration.merge(hbaseConf, plainHBaseConf);
+ } else {
+ // default to what is on the classpath
+ hbaseConf = HBaseConfiguration.create(conf);
+ }
+ return hbaseConf;
+ }
+
+ /**
+ * Given a row key prefix stored in a byte array, return a byte array for its
+ * immediate next row key.
+ *
+ * @param rowKeyPrefix The provided row key prefix, represented in an array.
+ * @return the closest next row key of the provided row key.
+ */
+ public static byte[] calculateTheClosestNextRowKeyForPrefix(
+ byte[] rowKeyPrefix) {
+ // Essentially we are treating it like an 'unsigned very very long' and
+ // doing +1 manually.
+ // Search for the place where the trailing 0xFFs start
+ int offset = rowKeyPrefix.length;
+ while (offset > 0) {
+ if (rowKeyPrefix[offset - 1] != (byte) 0xFF) {
+ break;
+ }
+ offset--;
+ }
+
+ if (offset == 0) {
+ // We got an 0xFFFF... (only FFs) stopRow value which is
+ // the last possible prefix before the end of the table.
+ // So set it to stop at the 'end of the table'
+ return HConstants.EMPTY_END_ROW;
+ }
+
+ // Copy the right length of the original
+ byte[] newStopRow = Arrays.copyOfRange(rowKeyPrefix, 0, offset);
+ // And increment the last one
+ newStopRow[newStopRow.length - 1]++;
+ return newStopRow;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTableRW.java
similarity index 97%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTableRW.java
index 027c8d5..984ef4f 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTableRW.java
@@ -28,7 +28,7 @@
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
/**
@@ -68,7 +68,7 @@
* |-------------------------------------------------------------------------|
*
*/
-public class EntityTable extends BaseTable {
+public class EntityTableRW extends BaseTableRW {
/** entity prefix. */
private static final String PREFIX =
YarnConfiguration.TIMELINE_SERVICE_PREFIX + "entity";
@@ -92,9 +92,9 @@
/** default max number of versions. */
private static final int DEFAULT_METRICS_MAX_VERSIONS = 1000;
- private static final Log LOG = LogFactory.getLog(EntityTable.class);
+ private static final Log LOG = LogFactory.getLog(EntityTableRW.class);
- public EntityTable() {
+ public EntityTableRW() {
super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);
}
@@ -102,7 +102,7 @@ public EntityTable() {
* (non-Javadoc)
*
* @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable
+ * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW#createTable
* (org.apache.hadoop.hbase.client.Admin,
* org.apache.hadoop.conf.Configuration)
*/
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTableRW.java
similarity index 96%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTableRW.java
index 8a0430c..5061e64 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTableRW.java
@@ -28,7 +28,7 @@
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
/**
* The flow activity table has column family info
@@ -51,7 +51,7 @@
* |-------------------------------------------|
*
*/
-public class FlowActivityTable extends BaseTable {
+public class FlowActivityTableRW extends BaseTableRW {
/** flow activity table prefix. */
private static final String PREFIX =
YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowactivity";
@@ -63,12 +63,12 @@
public static final String DEFAULT_TABLE_NAME =
"timelineservice.flowactivity";
- private static final Log LOG = LogFactory.getLog(FlowActivityTable.class);
+ private static final Log LOG = LogFactory.getLog(FlowActivityTableRW.class);
/** default max number of versions. */
public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE;
- public FlowActivityTable() {
+ public FlowActivityTableRW() {
super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);
}
@@ -76,7 +76,7 @@ public FlowActivityTable() {
* (non-Javadoc)
*
* @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable
+ * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW#createTable
* (org.apache.hadoop.hbase.client.Admin,
* org.apache.hadoop.conf.Configuration)
*/
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTableRW.java
similarity index 93%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTableRW.java
index ec973cb..151f991 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTableRW.java
@@ -28,9 +28,9 @@
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
/**
* The flow run table has column family info
@@ -85,7 +85,7 @@
* |-------------------------------------------|
*
*/
-public class FlowRunTable extends BaseTable {
+public class FlowRunTableRW extends BaseTableRW {
/** entity prefix. */
private static final String PREFIX =
YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowrun";
@@ -96,12 +96,12 @@
/** default value for flowrun table name. */
public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun";
- private static final Log LOG = LogFactory.getLog(FlowRunTable.class);
+ private static final Log LOG = LogFactory.getLog(FlowRunTableRW.class);
/** default max number of versions. */
public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE;
- public FlowRunTable() {
+ public FlowRunTableRW() {
super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);
}
@@ -109,7 +109,7 @@ public FlowRunTable() {
* (non-Javadoc)
*
* @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable
+ * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW#createTable
* (org.apache.hadoop.hbase.client.Admin,
* org.apache.hadoop.conf.Configuration)
*/
@@ -140,9 +140,9 @@ public void createTable(Admin admin, Configuration hbaseConf)
Path coprocessorJarPath = new Path(coprocessorJarPathStr);
LOG.info("CoprocessorJarPath=" + coprocessorJarPath.toString());
- flowRunTableDescp.addCoprocessor(
- FlowRunCoprocessor.class.getCanonicalName(), coprocessorJarPath,
- Coprocessor.PRIORITY_USER, null);
+ flowRunTableDescp.addCoprocessor("org.apache.hadoop.yarn.server." +
+ "timelineservice.storage.flow.FlowRunCoprocessor",
+ coprocessorJarPath, Coprocessor.PRIORITY_USER, null);
admin.createTable(flowRunTableDescp);
LOG.info("Status of table creation for " + table.getNameAsString() + "="
+ admin.tableExists(table));
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java
similarity index 90%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java
index 5bacf66..53ee4f2 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java
@@ -26,7 +26,8 @@
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumnPrefix;
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.apptoflow.AppToFlowTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefixRWHelper;
import org.apache.hadoop.yarn.webapp.NotFoundException;
/**
@@ -39,7 +40,7 @@
/**
* Used to look up the flow context.
*/
- private final AppToFlowTable appToFlowTable = new AppToFlowTable();
+ private final AppToFlowTableRW appToFlowTable = new AppToFlowTableRW();
public AbstractTimelineStorageReader(TimelineReaderContext ctxt) {
context = ctxt;
@@ -66,12 +67,12 @@ protected FlowContext lookupFlowContext(AppToFlowRowKey appToFlowRowKey,
Get get = new Get(rowKey);
Result result = appToFlowTable.getResult(hbaseConf, conn, get);
if (result != null && !result.isEmpty()) {
- Object flowName =
- AppToFlowColumnPrefix.FLOW_NAME.readResult(result, clusterId);
- Object flowRunId =
- AppToFlowColumnPrefix.FLOW_RUN_ID.readResult(result, clusterId);
- Object userId =
- AppToFlowColumnPrefix.USER_ID.readResult(result, clusterId);
+ Object flowName = ColumnPrefixRWHelper.readResult(
+ result, AppToFlowColumnPrefix.FLOW_NAME, clusterId);
+ Object flowRunId = ColumnPrefixRWHelper.readResult(
+ result, AppToFlowColumnPrefix.FLOW_RUN_ID, clusterId);
+ Object userId = ColumnPrefixRWHelper.readResult(
+ result, AppToFlowColumnPrefix.USER_ID, clusterId);
if (flowName == null || userId == null || flowRunId == null) {
throw new NotFoundException(
"Unable to find the context flow name, and flow run id, "
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
similarity index 97%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
index 4e8286d..e6adae2 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
@@ -47,9 +47,10 @@
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.application.ApplicationTableRW;
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.BaseTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnRWHelper;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
@@ -61,8 +62,8 @@
* application table.
*/
class ApplicationEntityReader extends GenericEntityReader {
- private static final ApplicationTable APPLICATION_TABLE =
- new ApplicationTable();
+ private static final ApplicationTableRW APPLICATION_TABLE =
+ new ApplicationTableRW();
public ApplicationEntityReader(TimelineReaderContext ctxt,
TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
@@ -75,9 +76,9 @@ public ApplicationEntityReader(TimelineReaderContext ctxt,
}
/**
- * Uses the {@link ApplicationTable}.
+ * Uses the {@link ApplicationTableRW}.
*/
- protected BaseTable> getTable() {
+ protected BaseTableRW> getTable() {
return APPLICATION_TABLE;
}
@@ -416,12 +417,14 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
}
TimelineEntity entity = new TimelineEntity();
entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
- String entityId = ApplicationColumn.ID.readResult(result).toString();
+ String entityId = ColumnRWHelper.readResult(result, ApplicationColumn.ID)
+ .toString();
entity.setId(entityId);
TimelineEntityFilters filters = getFilters();
// fetch created time
- Long createdTime = (Long) ApplicationColumn.CREATED_TIME.readResult(result);
+ Long createdTime = (Long) ColumnRWHelper.readResult(result,
+ ApplicationColumn.CREATED_TIME);
entity.setCreatedTime(createdTime);
EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
similarity index 98%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
index fd85878..64aab39 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
@@ -35,7 +35,7 @@
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
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.server.timelineservice.storage.entity.EntityTableRW;
import java.io.IOException;
import java.util.Arrays;
@@ -50,7 +50,7 @@
public final class EntityTypeReader extends AbstractTimelineStorageReader {
private static final Log LOG = LogFactory.getLog(EntityTypeReader.class);
- private static final EntityTable ENTITY_TABLE = new EntityTable();
+ private static final EntityTableRW ENTITY_TABLE = new EntityTableRW();
public EntityTypeReader(TimelineReaderContext context) {
super(context);
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java
similarity index 93%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java
index c741d0e..b2c26b2 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java
@@ -34,13 +34,14 @@
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.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefixRWHelper;
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 org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTableRW;
import com.google.common.base.Preconditions;
@@ -49,8 +50,8 @@
* flow activity table.
*/
class FlowActivityEntityReader extends TimelineEntityReader {
- private static final FlowActivityTable FLOW_ACTIVITY_TABLE =
- new FlowActivityTable();
+ private static final FlowActivityTableRW FLOW_ACTIVITY_TABLE =
+ new FlowActivityTableRW();
/**
* Used to convert Long key components to and from storage format.
@@ -69,10 +70,10 @@ public FlowActivityEntityReader(TimelineReaderContext ctxt,
}
/**
- * Uses the {@link FlowActivityTable}.
+ * Uses the {@link FlowActivityTableRW}.
*/
@Override
- protected BaseTable> getTable() {
+ protected BaseTableRW> getTable() {
return FLOW_ACTIVITY_TABLE;
}
@@ -143,8 +144,8 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
flowActivity.setId(flowActivity.getId());
// get the list of run ids along with the version that are associated with
// this flow on this day
- Map runIdsMap =
- FlowActivityColumnPrefix.RUN_ID.readResults(result, longKeyConverter);
+ Map runIdsMap = ColumnPrefixRWHelper.readResults(result,
+ FlowActivityColumnPrefix.RUN_ID, longKeyConverter);
for (Map.Entry e : runIdsMap.entrySet()) {
Long runId = e.getKey();
String version = (String)e.getValue();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
similarity index 95%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
index cedf96a..d370200 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
@@ -42,7 +42,8 @@
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnRWHelper;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
@@ -50,7 +51,7 @@
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.server.timelineservice.storage.flow.FlowRunTableRW;
import org.apache.hadoop.yarn.webapp.BadRequestException;
import com.google.common.base.Preconditions;
@@ -60,7 +61,7 @@
* table.
*/
class FlowRunEntityReader extends TimelineEntityReader {
- private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable();
+ private static final FlowRunTableRW FLOW_RUN_TABLE = new FlowRunTableRW();
public FlowRunEntityReader(TimelineReaderContext ctxt,
TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
@@ -73,10 +74,10 @@ public FlowRunEntityReader(TimelineReaderContext ctxt,
}
/**
- * Uses the {@link FlowRunTable}.
+ * Uses the {@link FlowRunTableRW}.
*/
@Override
- protected BaseTable> getTable() {
+ protected BaseTableRW> getTable() {
return FLOW_RUN_TABLE;
}
@@ -259,19 +260,22 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
}
// read the start time
- Long startTime = (Long) FlowRunColumn.MIN_START_TIME.readResult(result);
+ Long startTime = (Long) ColumnRWHelper.readResult(result,
+ FlowRunColumn.MIN_START_TIME);
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) ColumnRWHelper.readResult(result,
+ FlowRunColumn.MAX_END_TIME);
if (endTime != null) {
flowRun.setMaxEndTime(endTime.longValue());
}
// read the flow version
- String version = (String) FlowRunColumn.FLOW_VERSION.readResult(result);
+ String version = (String) ColumnRWHelper.readResult(result,
+ FlowRunColumn.FLOW_VERSION);
if (version != null) {
flowRun.setVersion(version);
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
similarity index 98%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
index f6904c5..c205ce8 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
@@ -43,8 +43,10 @@
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefixRWHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnRWHelper;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
@@ -55,7 +57,7 @@
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.server.timelineservice.storage.entity.EntityTableRW;
import com.google.common.base.Preconditions;
@@ -64,7 +66,7 @@
* table.
*/
class GenericEntityReader extends TimelineEntityReader {
- private static final EntityTable ENTITY_TABLE = new EntityTable();
+ private static final EntityTableRW ENTITY_TABLE = new EntityTableRW();
/**
* Used to convert strings key components to and from storage format.
@@ -83,9 +85,9 @@ public GenericEntityReader(TimelineReaderContext ctxt,
}
/**
- * Uses the {@link EntityTable}.
+ * Uses the {@link EntityTableRW}.
*/
- protected BaseTable> getTable() {
+ protected BaseTableRW> getTable() {
return ENTITY_TABLE;
}
@@ -523,7 +525,8 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
TimelineEntityFilters filters = getFilters();
// fetch created time
- Long createdTime = (Long) EntityColumn.CREATED_TIME.readResult(result);
+ Long createdTime = (Long) ColumnRWHelper.readResult(result,
+ EntityColumn.CREATED_TIME);
entity.setCreatedTime(createdTime);
EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
@@ -616,7 +619,7 @@ protected TimelineEntity parseEntity(Result result) throws IOException {
ColumnPrefix prefix, boolean isConfig) throws IOException {
// info and configuration are of type Map
Map columns =
- prefix.readResults(result, stringKeyConverter);
+ ColumnPrefixRWHelper.readResults(result, prefix, stringKeyConverter);
if (isConfig) {
for (Map.Entry column : columns.entrySet()) {
entity.addConfig(column.getKey(), column.getValue().toString());
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
similarity index 96%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
index 4c88cd3..7614455 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
@@ -45,8 +45,9 @@
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.BaseTableRW;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefixRWHelper;
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;
@@ -71,7 +72,7 @@
/**
* Main table the entity reader uses.
*/
- private BaseTable> table;
+ private BaseTableRW> table;
/**
* Used to convert strings key components to and from storage format.
@@ -260,7 +261,7 @@ public TimelineEntity readEntity(Configuration hbaseConf, Connection conn)
*
* @return A reference to the table.
*/
- protected BaseTable> getTable() {
+ protected BaseTableRW> getTable() {
return table;
}
@@ -313,8 +314,8 @@ protected abstract TimelineEntity parseEntity(Result result)
protected void readMetrics(TimelineEntity entity, Result result,
ColumnPrefix> columnPrefix) throws IOException {
NavigableMap> metricsResult =
- columnPrefix.readResultsWithTimestamps(
- result, stringKeyConverter);
+ ColumnPrefixRWHelper.readResultsWithTimestamps(result,
+ columnPrefix, stringKeyConverter);
for (Map.Entry> metricResult:
metricsResult.entrySet()) {
TimelineMetric metric = new TimelineMetric();
@@ -339,7 +340,7 @@ public boolean isSingleEntityRead() {
return singleEntityRead;
}
- protected void setTable(BaseTable> baseTable) {
+ protected void setTable(BaseTableRW> baseTable) {
this.table = baseTable;
}
@@ -404,8 +405,8 @@ protected boolean hasField(EnumSet fieldsToRetrieve,
protected void readRelationship(TimelineEntity entity, Result result,
ColumnPrefix prefix, boolean isRelatedTo) throws IOException {
// isRelatedTo and relatesTo are of type Map>
- Map columns =
- prefix.readResults(result, stringKeyConverter);
+ Map columns = ColumnPrefixRWHelper.readResults(result,
+ prefix, stringKeyConverter);
for (Map.Entry column : columns.entrySet()) {
for (String id : Separator.VALUES.splitEncoded(column.getValue()
.toString())) {
@@ -432,8 +433,8 @@ protected boolean hasField(EnumSet fieldsToRetrieve,
protected static void readEvents(TimelineEntity entity, Result result,
ColumnPrefix prefix) throws IOException {
Map eventsMap = new HashMap<>();
- Map eventsResult =
- prefix.readResults(result, new EventColumnNameConverter());
+ Map eventsResult = ColumnPrefixRWHelper
+ .readResults(result, prefix, new EventColumnNameConverter());
for (Map.Entry
eventResult : eventsResult.entrySet()) {
EventColumnName eventColumnName = eventResult.getKey();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/pom.xml
new file mode 100644
index 0000000..fface3d
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/pom.xml
@@ -0,0 +1,128 @@
+
+
+
+ hadoop-yarn-server-timelineservice-hbase
+ org.apache.hadoop
+ 3.0.0-alpha2-SNAPSHOT
+
+ 4.0.0
+
+ hadoop-yarn-server-timelineservice-hbase-schema
+
+
+
+ com.google.guava
+ guava
+
+
+
+ org.apache.hadoop
+ hadoop-annotations
+
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-api
+
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-applicationhistoryservice
+
+
+
+ org.apache.hbase
+ hbase-common
+
+
+ hadoop-common
+ org.apache.hadoop
+
+
+ org.apache.hadoop
+ hadoop-mapreduce-client-core
+
+
+ org.mortbay.jetty
+ jetty-util
+
+
+
+
+
+ org.apache.hbase
+ hbase-client
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+ org.apache.hadoop
+ hadoop-auth
+
+
+ org.apache.hadoop
+ hadoop-mapreduce-client-core
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ test
+
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ test-jar
+ test
+
+
+
+ junit
+ junit
+ test
+
+
+
+
+
+
+
+ maven-jar-plugin
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+
+ org.apache.maven.plugins
+ maven-javadoc-plugin
+
+
+
+ junit
+ junit
+ 4.11
+
+
+
+
+
+
+
+
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java
new file mode 100644
index 0000000..7588fba
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java
@@ -0,0 +1,145 @@
+/**
+ * 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.Column;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
+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.ValueConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Identifies fully qualified columns for the {@link ApplicationTable}.
+ */
+public enum ApplicationColumn implements Column {
+
+ /**
+ * App id.
+ */
+ ID(ApplicationColumnFamily.INFO, "id"),
+
+ /**
+ * When the application was created.
+ */
+ CREATED_TIME(ApplicationColumnFamily.INFO, "created_time",
+ new LongConverter()),
+
+ /**
+ * The version of the flow that this app belongs to.
+ */
+ FLOW_VERSION(ApplicationColumnFamily.INFO, "flow_version");
+
+ private final ColumnFamily columnFamily;
+ private final String columnQualifier;
+ private final byte[] columnQualifierBytes;
+ private final ValueConverter converter;
+
+ private ApplicationColumn(ColumnFamily columnFamily,
+ String columnQualifier) {
+ this(columnFamily, columnQualifier, GenericConverter.getInstance());
+ }
+
+ private ApplicationColumn(ColumnFamily columnFamily,
+ String columnQualifier, ValueConverter converter) {
+ this.columnFamily = columnFamily;
+ this.columnQualifier = columnQualifier;
+ // Future-proof by ensuring the right column prefix hygiene.
+ this.columnQualifierBytes =
+ Bytes.toBytes(Separator.SPACE.encode(columnQualifier));
+ this.converter = converter;
+ }
+
+ /**
+ * @return the column name value
+ */
+ private String getColumnQualifier() {
+ return columnQualifier;
+ }
+
+ @Override
+ public byte[] getColumnQualifierBytes() {
+ return columnQualifierBytes.clone();
+ }
+
+ @Override
+ public byte[] getColumnFamilyBytes() {
+ return columnFamily.getBytes();
+ }
+
+ @Override
+ public ValueConverter getValueConverter() {
+ return converter;
+ }
+
+ @Override
+ public Attribute[] combineAttrsWithAggr(Attribute... attributes) {
+ return attributes;
+ }
+
+ /**
+ * Retrieve an {@link ApplicationColumn} given a name, or null if there is no
+ * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
+ * and only if {@code x.equals(y)} or {@code (x == y == null)}.
+ *
+ * @param columnQualifier Name of the column to retrieve
+ * @return the corresponding {@link ApplicationColumn} or null
+ */
+ public static final ApplicationColumn columnFor(String columnQualifier) {
+
+ // Match column based on value, assume column family matches.
+ for (ApplicationColumn ac : ApplicationColumn.values()) {
+ // Find a match based only on name.
+ if (ac.getColumnQualifier().equals(columnQualifier)) {
+ return ac;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+ /**
+ * Retrieve an {@link ApplicationColumn} given a name, or null if there is no
+ * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
+ * if and only if {@code a.equals(b) & x.equals(y)} or
+ * {@code (x == y == null)}
+ *
+ * @param columnFamily The columnFamily for which to retrieve the column.
+ * @param name Name of the column to retrieve
+ * @return the corresponding {@link ApplicationColumn} or null if both
+ * arguments don't match.
+ */
+ public static final ApplicationColumn columnFor(
+ ApplicationColumnFamily columnFamily, String name) {
+
+ for (ApplicationColumn ac : ApplicationColumn.values()) {
+ // Find a match based column family and on name.
+ if (ac.columnFamily.equals(columnFamily)
+ && ac.getColumnQualifier().equals(name)) {
+ return ac;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
new file mode 100644
index 0000000..6e66ce1
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
@@ -0,0 +1,198 @@
+/**
+ * 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.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+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.ValueConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Identifies partially qualified columns for the application table.
+ */
+public enum ApplicationColumnPrefix implements ColumnPrefix {
+
+ /**
+ * To store TimelineEntity getIsRelatedToEntities values.
+ */
+ IS_RELATED_TO(ApplicationColumnFamily.INFO, "s"),
+
+ /**
+ * To store TimelineEntity getRelatesToEntities values.
+ */
+ RELATES_TO(ApplicationColumnFamily.INFO, "r"),
+
+ /**
+ * To store TimelineEntity info values.
+ */
+ INFO(ApplicationColumnFamily.INFO, "i"),
+
+ /**
+ * Lifecycle events for an application.
+ */
+ EVENT(ApplicationColumnFamily.INFO, "e"),
+
+ /**
+ * Config column stores configuration with config key as the column name.
+ */
+ CONFIG(ApplicationColumnFamily.CONFIGS, null),
+
+ /**
+ * Metrics are stored with the metric name as the column name.
+ */
+ METRIC(ApplicationColumnFamily.METRICS, null, new LongConverter());
+
+ private final ColumnFamily columnFamily;
+
+ /**
+ * Can be null for those cases where the provided column qualifier is the
+ * entire column name.
+ */
+ private final String columnPrefix;
+ private final byte[] columnPrefixBytes;
+ private final ValueConverter converter;
+
+ /**
+ * Private constructor, meant to be used by the enum definition.
+ *
+ * @param columnFamily that this column is stored in.
+ * @param columnPrefix for this column.
+ */
+ private ApplicationColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix) {
+ this(columnFamily, columnPrefix, GenericConverter.getInstance());
+ }
+
+ /**
+ * Private constructor, meant to be used by the enum definition.
+ *
+ * @param columnFamily that this column is stored in.
+ * @param columnPrefix for this column.
+ * @param converter used to encode/decode values to be stored in HBase for
+ * this column prefix.
+ */
+ private ApplicationColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix, ValueConverter converter) {
+ this.columnFamily = columnFamily;
+ this.columnPrefix = columnPrefix;
+ if (columnPrefix == null) {
+ this.columnPrefixBytes = null;
+ } else {
+ // Future-proof by ensuring the right column prefix hygiene.
+ this.columnPrefixBytes =
+ Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
+ }
+ this.converter = converter;
+ }
+
+ /**
+ * @return the column name value
+ */
+ private String getColumnPrefix() {
+ return columnPrefix;
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(
+ this.columnPrefixBytes, qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(String qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(
+ this.columnPrefixBytes, qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes() {
+ return columnPrefixBytes != null ? columnPrefixBytes.clone() : null;
+ }
+
+ @Override
+ public byte[] getColumnFamilyBytes() {
+ return columnFamily.getBytes();
+ }
+
+ @Override
+ public Attribute[] combineAttrsWithAggr(Attribute... attributes) {
+ return attributes;
+ }
+
+ public ValueConverter getValueConverter() {
+ return converter;
+ }
+
+ /**
+ * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there
+ * is no match. The following holds true: {@code columnFor(x) == columnFor(y)}
+ * if and only if {@code x.equals(y)} or {@code (x == y == null)}
+ *
+ * @param columnPrefix Name of the column to retrieve
+ * @return the corresponding {@link ApplicationColumnPrefix} or null
+ */
+ public static final ApplicationColumnPrefix columnFor(String columnPrefix) {
+
+ // Match column based on value, assume column family matches.
+ for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) {
+ // Find a match based only on name.
+ if (acp.getColumnPrefix().equals(columnPrefix)) {
+ return acp;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+ /**
+ * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there
+ * is no match. The following holds true:
+ * {@code columnFor(a,x) == columnFor(b,y)} if and only if
+ * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
+ *
+ * @param columnFamily The columnFamily for which to retrieve the column.
+ * @param columnPrefix Name of the column to retrieve
+ * @return the corresponding {@link ApplicationColumnPrefix} or null if both
+ * arguments don't match.
+ */
+ public static final ApplicationColumnPrefix columnFor(
+ ApplicationColumnFamily columnFamily, String columnPrefix) {
+
+ // TODO: needs unit test to confirm and need to update javadoc to explain
+ // null prefix case.
+
+ for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) {
+ // Find a match based column family and on name.
+ if (acp.columnFamily.equals(columnFamily)
+ && (((columnPrefix == null) && (acp.getColumnPrefix() == null)) ||
+ (acp.getColumnPrefix().equals(columnPrefix)))) {
+ return acp;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
new file mode 100644
index 0000000..0374053
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
@@ -0,0 +1,6 @@
+package org.apache.hadoop.yarn.server.timelineservice.storage.application;
+
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+
+public class ApplicationTable implements BaseTable{
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java
new file mode 100644
index 0000000..03f508f
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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
+ * contains classes related to implementation for application table.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.application;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
new file mode 100644
index 0000000..462b4ae
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
@@ -0,0 +1,152 @@
+/**
+ * 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.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Identifies partially qualified columns for the app-to-flow table.
+ */
+public enum AppToFlowColumnPrefix implements ColumnPrefix {
+
+ /**
+ * The flow name.
+ */
+ FLOW_NAME(AppToFlowColumnFamily.MAPPING, "flow_name"),
+
+ /**
+ * The flow run ID.
+ */
+ FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "flow_run_id"),
+
+ /**
+ * The user.
+ */
+ USER_ID(AppToFlowColumnFamily.MAPPING, "user_id");
+
+ private final ColumnFamily columnFamily;
+ private final String columnPrefix;
+ private final byte[] columnPrefixBytes;
+ private final ValueConverter converter;
+
+ private AppToFlowColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix) {
+ this.columnFamily = columnFamily;
+ this.columnPrefix = columnPrefix;
+ if (columnPrefix == null) {
+ this.columnPrefixBytes = null;
+ } else {
+ // Future-proof by ensuring the right column prefix hygiene.
+ this.columnPrefixBytes =
+ Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
+ }
+ this.converter = GenericConverter.getInstance();
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes() {
+ return columnPrefixBytes != null ? columnPrefixBytes.clone() : null;
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(String qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(
+ columnPrefixBytes, qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(
+ columnPrefixBytes, qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnFamilyBytes() {
+ return columnFamily.getBytes();
+ }
+
+ @Override
+ public Attribute[] combineAttrsWithAggr(Attribute... attributes) {
+ return attributes;
+ }
+
+ @Override
+ public ValueConverter getValueConverter() {
+ return converter;
+ }
+
+ /**
+ * Retrieve an {@link AppToFlowColumnPrefix} given a name, or null if there
+ * is no match. The following holds true: {@code columnFor(x) == columnFor(y)}
+ * if and only if {@code x.equals(y)} or {@code (x == y == null)}
+ *
+ * @param columnPrefix Name of the column to retrieve
+ * @return the corresponding {@link AppToFlowColumnPrefix} or null
+ */
+ public static final AppToFlowColumnPrefix columnFor(String columnPrefix) {
+
+ // Match column based on value, assume column family matches.
+ for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) {
+ // Find a match based only on name.
+ if (afcp.columnPrefix.equals(columnPrefix)) {
+ return afcp;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+ /**
+ * Retrieve an {@link AppToFlowColumnPrefix} given a name, or null if there
+ * is no match. The following holds true:
+ * {@code columnFor(a,x) == columnFor(b,y)} if and only if
+ * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
+ *
+ * @param columnFamily The columnFamily for which to retrieve the column.
+ * @param columnPrefix Name of the column to retrieve
+ * @return the corresponding {@link AppToFlowColumnPrefix} or null if both
+ * arguments don't match.
+ */
+ public static final AppToFlowColumnPrefix columnFor(
+ AppToFlowColumnFamily columnFamily, String columnPrefix) {
+
+ // TODO: needs unit test to confirm and need to update javadoc to explain
+ // null prefix case.
+
+ for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) {
+ // Find a match based column family and on name.
+ if (afcp.columnFamily.equals(columnFamily)
+ && (((columnPrefix == null) && (afcp.columnPrefix == null)) ||
+ (afcp.columnPrefix.equals(columnPrefix)))) {
+ return afcp;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
new file mode 100644
index 0000000..a37a7a9
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
@@ -0,0 +1,6 @@
+package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
+
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+
+public class AppToFlowTable implements BaseTable {
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java
new file mode 100644
index 0000000..f01d982
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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
+ * contains classes related to implementation for app to flow table.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
similarity index 97%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
index c165801..e0e91a6 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
@@ -55,7 +55,7 @@ public AppIdKeyConverter() {
LongConverter.invertLong(appId.getClusterTimestamp()));
System.arraycopy(clusterTs, 0, appIdBytes, 0, Bytes.SIZEOF_LONG);
byte[] seqId = Bytes.toBytes(
- HBaseTimelineStorageUtils.invertInt(appId.getId()));
+ HBaseTimelineSchemaUtils.invertInt(appId.getId()));
System.arraycopy(seqId, 0, appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT);
return appIdBytes;
}
@@ -80,7 +80,7 @@ public String decode(byte[] appIdBytes) {
}
long clusterTs = LongConverter.invertLong(
Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
- int seqId = HBaseTimelineStorageUtils.invertInt(
+ int seqId = HBaseTimelineSchemaUtils.invertInt(
Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));
return ApplicationId.newInstance(clusterTs, seqId).toString();
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java
new file mode 100644
index 0000000..f42cdf7
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java
@@ -0,0 +1,7 @@
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+/**
+ * A table type for type safety.
+ */
+public interface BaseTable {
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java
new file mode 100644
index 0000000..75a7c40
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java
@@ -0,0 +1,51 @@
+/**
+ * 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 org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * A Column represents the way to store a fully qualified column in a specific
+ * table.
+ */
+public interface Column {
+ /**
+ * Returns column family name(as bytes) associated with this column.
+ * @return a byte array encoding column family for this column qualifier.
+ */
+ byte[] getColumnFamilyBytes();
+
+ /**
+ * Get byte representation for this column qualifier.
+ * @return a byte array representing column qualifier.
+ */
+ byte[] getColumnQualifierBytes();
+
+ /**
+ * Returns value converter implementation associated with this column.
+ * @return a {@link ValueConverter} implementation.
+ */
+ ValueConverter getValueConverter();
+
+
+ /**
+ * Return attributed combined with aggregations, if any
+ * @return an array of Attributes
+ */
+ Attribute[] combineAttrsWithAggr(Attribute... attributes);
+}
\ No newline at end of file
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
new file mode 100644
index 0000000..9d31b24
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
@@ -0,0 +1,104 @@
+/**
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+/**
+ * This class is meant to be used only by explicit Columns, and not directly to
+ * write by clients.
+ *
+ * @param refers to the table.
+ */
+public final class ColumnHelper {
+ private static final Log LOG = LogFactory.getLog(ColumnHelper.class);
+
+ private ColumnHelper() {
+ }
+
+ /**
+ * @param columnPrefixBytes The byte representation for the column prefix.
+ * Should not contain {@link Separator#QUALIFIERS}.
+ * @param qualifier for the remainder of the column.
+ * {@link Separator#QUALIFIERS} is permissible in the qualifier
+ * as it is joined only with the column prefix bytes.
+ * @return fully sanitized column qualifier that is a combination of prefix
+ * and qualifier. If prefix is null, the result is simply the encoded
+ * qualifier without any separator.
+ */
+ public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
+ String qualifier) {
+
+ // We don't want column names to have spaces / tabs.
+ byte[] encodedQualifier =
+ Separator.encode(qualifier, Separator.SPACE, Separator.TAB);
+ if (columnPrefixBytes == null) {
+ return encodedQualifier;
+ }
+
+ // Convert qualifier to lower case, strip of separators and tag on column
+ // prefix.
+ byte[] columnQualifier =
+ Separator.QUALIFIERS.join(columnPrefixBytes, encodedQualifier);
+ return columnQualifier;
+ }
+
+ /**
+ * @param columnPrefixBytes The byte representation for the column prefix.
+ * Should not contain {@link Separator#QUALIFIERS}.
+ * @param qualifier for the remainder of the column.
+ * @return fully sanitized column qualifier that is a combination of prefix
+ * and qualifier. If prefix is null, the result is simply the encoded
+ * qualifier without any separator.
+ */
+ public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
+ long qualifier) {
+
+ if (columnPrefixBytes == null) {
+ return Bytes.toBytes(qualifier);
+ }
+
+ // Convert qualifier to lower case, strip of separators and tag on column
+ // prefix.
+ byte[] columnQualifier =
+ Separator.QUALIFIERS.join(columnPrefixBytes, Bytes.toBytes(qualifier));
+ return columnQualifier;
+ }
+
+ /**
+ * @param columnPrefixBytes The byte representation for the column prefix.
+ * Should not contain {@link Separator#QUALIFIERS}.
+ * @param qualifier the byte representation for the remainder of the column.
+ * @return fully sanitized column qualifier that is a combination of prefix
+ * and qualifier. If prefix is null, the result is simply the encoded
+ * qualifier without any separator.
+ */
+ public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
+ byte[] qualifier) {
+
+ if (columnPrefixBytes == null) {
+ return qualifier;
+ }
+
+ byte[] columnQualifier =
+ Separator.QUALIFIERS.join(columnPrefixBytes, qualifier);
+ return columnQualifier;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java
new file mode 100644
index 0000000..7e506f0
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java
@@ -0,0 +1,65 @@
+/**
+ * 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 org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Used to represent a partially qualified column, where the actual column name
+ * will be composed of a prefix and the remainder of the column qualifier. The
+ * prefix can be null, in which case the column qualifier will be completely
+ * determined when the values are stored.
+ */
+public interface ColumnPrefix {
+
+ /**
+ * Return the column prefix in bytes.
+ * @return a byte array encoding the column prefix
+ */
+ byte[] getColumnPrefixBytes();
+
+ /**
+ * @param qualifierPrefix Column qualifier or prefix of qualifier.
+ * @return a byte array encoding column prefix and qualifier/prefix passed.
+ */
+ byte[] getColumnPrefixBytes(String qualifierPrefix);
+
+ /**
+ * @param qualifierPrefix Column qualifier or prefix of qualifier.
+ * @return a byte array encoding column prefix and qualifier/prefix passed.
+ */
+ byte[] getColumnPrefixBytes(byte[] qualifierPrefix);
+
+ /**
+ * Returns column family name(as bytes) associated with this column prefix.
+ * @return a byte array encoding column family for this prefix.
+ */
+ byte[] getColumnFamilyBytes();
+
+ /**
+ * Returns value converter implementation associated with this column prefix.
+ * @return a {@link ValueConverter} implementation.
+ */
+ ValueConverter getValueConverter();
+
+ /**
+ * Return attributed combined with aggregations, if any
+ * @return an array of Attributes
+ */
+ Attribute[] combineAttrsWithAggr(Attribute... attributes);
+}
\ No newline at end of file
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineSchemaUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineSchemaUtils.java
new file mode 100644
index 0000000..b427dda
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineSchemaUtils.java
@@ -0,0 +1,210 @@
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.util.Bytes;
+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;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+public final class HBaseTimelineSchemaUtils {
+ /** milliseconds in one day. */
+ public static final long MILLIS_ONE_DAY = 86400000L;
+
+ private HBaseTimelineSchemaUtils() {
+ }
+
+
+ /**
+ * Combines the input array of attributes and the input aggregation operation
+ * into a new array of attributes.
+ *
+ * @param attributes Attributes to be combined.
+ * @param aggOp Aggregation operation.
+ * @return array of combined attributes.
+ */
+ public static Attribute[] combineAttributes(Attribute[] attributes,
+ AggregationOperation aggOp) {
+ int newLength = getNewLengthCombinedAttributes(attributes, aggOp);
+ Attribute[] combinedAttributes = new Attribute[newLength];
+
+ if (attributes != null) {
+ System.arraycopy(attributes, 0, combinedAttributes, 0, attributes.length);
+ }
+
+ if (aggOp != null) {
+ Attribute a2 = aggOp.getAttribute();
+ combinedAttributes[newLength - 1] = a2;
+ }
+ return combinedAttributes;
+ }
+
+ /**
+ * Returns a number for the new array size. The new array is the combination
+ * of input array of attributes and the input aggregation operation.
+ *
+ * @param attributes Attributes.
+ * @param aggOp Aggregation operation.
+ * @return the size for the new array
+ */
+ private static int getNewLengthCombinedAttributes(Attribute[] attributes,
+ AggregationOperation aggOp) {
+ int oldLength = getAttributesLength(attributes);
+ int aggLength = getAppOpLength(aggOp);
+ return oldLength + aggLength;
+ }
+
+ private static int getAppOpLength(AggregationOperation aggOp) {
+ if (aggOp != null) {
+ return 1;
+ }
+ return 0;
+ }
+
+ private static int getAttributesLength(Attribute[] attributes) {
+ if (attributes != null) {
+ return attributes.length;
+ }
+ return 0;
+ }
+
+ /**
+ * Returns the first seen aggregation operation as seen in the list of input
+ * tags or null otherwise.
+ *
+ * @param tags list of HBase tags.
+ * @return AggregationOperation
+ */
+ public static AggregationOperation getAggregationOperationFromTagsList(
+ List tags) {
+ for (AggregationOperation aggOp : AggregationOperation.values()) {
+ for (Tag tag : tags) {
+ if (tag.getType() == aggOp.getTagType()) {
+ return aggOp;
+ }
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Creates a {@link Tag} from the input attribute.
+ *
+ * @param attribute Attribute from which tag has to be fetched.
+ * @return a HBase Tag.
+ */
+ public static Tag getTagFromAttribute(Map.Entry attribute) {
+ // attribute could be either an Aggregation Operation or
+ // an Aggregation Dimension
+ // Get the Tag type from either
+ AggregationOperation aggOp = AggregationOperation
+ .getAggregationOperation(attribute.getKey());
+ if (aggOp != null) {
+ Tag t = new Tag(aggOp.getTagType(), attribute.getValue());
+ return t;
+ }
+
+ AggregationCompactionDimension aggCompactDim =
+ AggregationCompactionDimension.getAggregationCompactionDimension(
+ attribute.getKey());
+ if (aggCompactDim != null) {
+ Tag t = new Tag(aggCompactDim.getTagType(), attribute.getValue());
+ return t;
+ }
+ return null;
+ }
+
+ /**
+ * creates a new cell based on the input cell but with the new value.
+ *
+ * @param origCell Original cell
+ * @param newValue new cell value
+ * @return cell
+ * @throws IOException while creating new cell.
+ */
+ public static Cell createNewCell(Cell origCell, byte[] newValue)
+ throws IOException {
+ return CellUtil.createCell(CellUtil.cloneRow(origCell),
+ CellUtil.cloneFamily(origCell), CellUtil.cloneQualifier(origCell),
+ origCell.getTimestamp(), KeyValue.Type.Put.getCode(), newValue);
+ }
+
+ /**
+ * creates a cell with the given inputs.
+ *
+ * @param row row of the cell to be created
+ * @param family column family name of the new cell
+ * @param qualifier qualifier for the new cell
+ * @param ts timestamp of the new cell
+ * @param newValue value of the new cell
+ * @param tags tags in the new cell
+ * @return cell
+ * @throws IOException while creating the cell.
+ */
+ public static Cell createNewCell(byte[] row, byte[] family, byte[] qualifier,
+ long ts, byte[] newValue, byte[] tags) throws IOException {
+ return CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put,
+ newValue, tags);
+ }
+
+ /**
+ * returns app id from the list of tags.
+ *
+ * @param tags cell tags to be looked into
+ * @return App Id as the AggregationCompactionDimension
+ */
+ public static String getAggregationCompactionDimension(List tags) {
+ String appId = null;
+ for (Tag t : tags) {
+ if (AggregationCompactionDimension.APPLICATION_ID.getTagType() == t
+ .getType()) {
+ appId = Bytes.toString(t.getValue());
+ return appId;
+ }
+ }
+ return appId;
+ }
+
+ /**
+ * 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
+ * (scans start at the largest int first).
+ *
+ * @param key value to be inverted so that the latest version will be first in
+ * a scan.
+ * @return inverted int
+ */
+ public static int invertInt(int key) {
+ return Integer.MAX_VALUE - key;
+ }
+
+ /**
+ * returns the timestamp of that day's start (which is midnight 00:00:00 AM)
+ * for a given input timestamp.
+ *
+ * @param ts Timestamp.
+ * @return timestamp of that day's beginning (midnight)
+ */
+ public static long getTopOfTheDayTimestamp(long ts) {
+ long dayTimestamp = ts - (ts % MILLIS_ONE_DAY);
+ return dayTimestamp;
+ }
+
+ /**
+ * Checks if passed object is of integral type(Short/Integer/Long).
+ *
+ * @param obj Object to be checked.
+ * @return true if object passed is of type Short or Integer or Long, false
+ * otherwise.
+ */
+ public static boolean isIntegralValue(Object obj) {
+ return (obj instanceof Short) || (obj instanceof Integer) ||
+ (obj instanceof Long);
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java
similarity index 97%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java
index 6ab69f7..0857980 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java
@@ -40,7 +40,7 @@ public LongConverter() {
@Override
public byte[] encodeValue(Object value) throws IOException {
- if (!HBaseTimelineStorageUtils.isIntegralValue(value)) {
+ if (!HBaseTimelineSchemaUtils.isIntegralValue(value)) {
throw new IOException("Expected integral value");
}
return Bytes.toBytes(((Number)value).longValue());
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java
new file mode 100644
index 0000000..dcccf74
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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 contains
+ * classes which define and implement reading and writing to backend storage.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
new file mode 100644
index 0000000..15c1a55
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
@@ -0,0 +1,149 @@
+/**
+ * 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.Column;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
+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.ValueConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Identifies fully qualified columns for the {@link EntityTable}.
+ */
+public enum EntityColumn implements Column {
+
+ /**
+ * Identifier for the entity.
+ */
+ ID(EntityColumnFamily.INFO, "id"),
+
+ /**
+ * The type of entity.
+ */
+ TYPE(EntityColumnFamily.INFO, "type"),
+
+ /**
+ * When the entity was created.
+ */
+ CREATED_TIME(EntityColumnFamily.INFO, "created_time", new LongConverter()),
+
+ /**
+ * The version of the flow that this entity belongs to.
+ */
+ FLOW_VERSION(EntityColumnFamily.INFO, "flow_version");
+
+ private final ColumnFamily columnFamily;
+ private final String columnQualifier;
+ private final byte[] columnQualifierBytes;
+ private final ValueConverter converter;
+
+ EntityColumn(ColumnFamily columnFamily,
+ String columnQualifier) {
+ this(columnFamily, columnQualifier, GenericConverter.getInstance());
+ }
+
+ EntityColumn(ColumnFamily columnFamily,
+ String columnQualifier, ValueConverter converter) {
+ this.columnFamily = columnFamily;
+ this.columnQualifier = columnQualifier;
+ // Future-proof by ensuring the right column prefix hygiene.
+ this.columnQualifierBytes =
+ Bytes.toBytes(Separator.SPACE.encode(columnQualifier));
+ this.converter = converter;
+ }
+
+ /**
+ * @return the column name value
+ */
+ private String getColumnQualifier() {
+ return columnQualifier;
+ }
+
+ @Override
+ public Attribute[] combineAttrsWithAggr(Attribute... attributes) {
+ return attributes;
+ }
+
+ /**
+ * Retrieve an {@link EntityColumn} given a name, or null if there is no
+ * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
+ * and only if {@code x.equals(y)} or {@code (x == y == null)}
+ *
+ * @param columnQualifier Name of the column to retrieve
+ * @return the corresponding {@link EntityColumn} or null
+ */
+ public static final EntityColumn columnFor(String columnQualifier) {
+
+ // Match column based on value, assume column family matches.
+ for (EntityColumn ec : EntityColumn.values()) {
+ // Find a match based only on name.
+ if (ec.getColumnQualifier().equals(columnQualifier)) {
+ return ec;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+ @Override
+ public byte[] getColumnQualifierBytes() {
+ return columnQualifierBytes.clone();
+ }
+
+ @Override
+ public byte[] getColumnFamilyBytes() {
+ return columnFamily.getBytes();
+ }
+
+ @Override
+ public ValueConverter getValueConverter() {
+ return converter;
+ }
+
+ /**
+ * Retrieve an {@link EntityColumn} given a name, or null if there is no
+ * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
+ * if and only if {@code a.equals(b) & x.equals(y)} or
+ * {@code (x == y == null)}
+ *
+ * @param columnFamily The columnFamily for which to retrieve the column.
+ * @param name Name of the column to retrieve
+ * @return the corresponding {@link EntityColumn} or null if both arguments
+ * don't match.
+ */
+ public static final EntityColumn columnFor(EntityColumnFamily columnFamily,
+ String name) {
+
+ for (EntityColumn ec : EntityColumn.values()) {
+ // Find a match based column family and on name.
+ if (ec.columnFamily.equals(columnFamily)
+ && ec.getColumnQualifier().equals(name)) {
+ return ec;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
new file mode 100644
index 0000000..30c09d8
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
@@ -0,0 +1,209 @@
+/**
+ * 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.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+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.ValueConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Identifies partially qualified columns for the entity table.
+ */
+public enum EntityColumnPrefix implements ColumnPrefix {
+
+ /**
+ * To store TimelineEntity getIsRelatedToEntities values.
+ */
+ IS_RELATED_TO(EntityColumnFamily.INFO, "s"),
+
+ /**
+ * To store TimelineEntity getRelatesToEntities values.
+ */
+ RELATES_TO(EntityColumnFamily.INFO, "r"),
+
+ /**
+ * To store TimelineEntity info values.
+ */
+ INFO(EntityColumnFamily.INFO, "i"),
+
+ /**
+ * Lifecycle events for an entity.
+ */
+ EVENT(EntityColumnFamily.INFO, "e", true),
+
+ /**
+ * Config column stores configuration with config key as the column name.
+ */
+ CONFIG(EntityColumnFamily.CONFIGS, null),
+
+ /**
+ * Metrics are stored with the metric name as the column name.
+ */
+ METRIC(EntityColumnFamily.METRICS, null, new LongConverter());
+
+ private final ColumnFamily columnFamily;
+
+ /**
+ * Can be null for those cases where the provided column qualifier is the
+ * entire column name.
+ */
+ private final String columnPrefix;
+ private final byte[] columnPrefixBytes;
+ private final ValueConverter converter;
+
+ /**
+ * Private constructor, meant to be used by the enum definition.
+ *
+ * @param columnFamily that this column is stored in.
+ * @param columnPrefix for this column.
+ */
+ EntityColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix) {
+ this(columnFamily, columnPrefix, false, GenericConverter.getInstance());
+ }
+
+ EntityColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix, boolean compondColQual) {
+ this(columnFamily, columnPrefix, compondColQual,
+ GenericConverter.getInstance());
+ }
+
+ EntityColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix, ValueConverter converter) {
+ this(columnFamily, columnPrefix, false, converter);
+ }
+
+ /**
+ * Private constructor, meant to be used by the enum definition.
+ *
+ * @param columnFamily that this column is stored in.
+ * @param columnPrefix for this column.
+ * @param converter used to encode/decode values to be stored in HBase for
+ * this column prefix.
+ */
+ EntityColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix, boolean compondColQual, ValueConverter converter) {
+ this.columnFamily = columnFamily;
+ this.columnPrefix = columnPrefix;
+ if (columnPrefix == null) {
+ this.columnPrefixBytes = null;
+ } else {
+ // Future-proof by ensuring the right column prefix hygiene.
+ this.columnPrefixBytes =
+ Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
+ }
+ this.converter = converter;
+ }
+
+ /**
+ * @return the column name value
+ */
+ public String getColumnPrefix() {
+ return columnPrefix;
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(
+ this.columnPrefixBytes, qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(String qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(
+ this.columnPrefixBytes, qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes() {
+ return columnPrefixBytes != null ? columnPrefixBytes.clone() : null;
+ }
+
+ @Override
+ public byte[] getColumnFamilyBytes() {
+ return columnFamily.getBytes();
+ }
+
+ @Override
+ public ValueConverter getValueConverter() {
+ return converter;
+ }
+
+ @Override
+ public Attribute[] combineAttrsWithAggr(Attribute... attributes) {
+ return attributes;
+ }
+
+ /**
+ * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no
+ * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
+ * and only if {@code x.equals(y)} or {@code (x == y == null)}
+ *
+ * @param columnPrefix Name of the column to retrieve
+ * @return the corresponding {@link EntityColumnPrefix} or null
+ */
+ public static final EntityColumnPrefix columnFor(String columnPrefix) {
+
+ // Match column based on value, assume column family matches.
+ for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) {
+ // Find a match based only on name.
+ if (ecp.getColumnPrefix().equals(columnPrefix)) {
+ return ecp;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+ /**
+ * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no
+ * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
+ * if and only if {@code (x == y == null)} or
+ * {@code a.equals(b) & x.equals(y)}
+ *
+ * @param columnFamily The columnFamily for which to retrieve the column.
+ * @param columnPrefix Name of the column to retrieve
+ * @return the corresponding {@link EntityColumnPrefix} or null if both
+ * arguments don't match.
+ */
+ public static final EntityColumnPrefix columnFor(
+ EntityColumnFamily columnFamily, String columnPrefix) {
+
+ // TODO: needs unit test to confirm and need to update javadoc to explain
+ // null prefix case.
+
+ for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) {
+ // Find a match based column family and on name.
+ if (ecp.columnFamily.equals(columnFamily)
+ && (((columnPrefix == null) && (ecp.getColumnPrefix() == null)) ||
+ (ecp.getColumnPrefix().equals(columnPrefix)))) {
+ return ecp;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
new file mode 100644
index 0000000..74baf61
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
@@ -0,0 +1,6 @@
+package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
+
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+
+public class EntityTable implements BaseTable {
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java
new file mode 100644
index 0000000..bb0e331
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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
+ * contains classes related to implementation for entity table.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
new file mode 100644
index 0000000..bb736c1
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
@@ -0,0 +1,157 @@
+/**
+ * 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.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
+
+/**
+ * Identifies partially qualified columns for the {@link FlowActivityTable}.
+ */
+public enum FlowActivityColumnPrefix
+ implements ColumnPrefix {
+
+ /**
+ * To store run ids of the flows.
+ */
+ RUN_ID(FlowActivityColumnFamily.INFO, "r", null);
+
+ private final ColumnFamily columnFamily;
+
+ /**
+ * Can be null for those cases where the provided column qualifier is the
+ * entire column name.
+ */
+ private final String columnPrefix;
+ private final byte[] columnPrefixBytes;
+ private final ValueConverter converter;
+
+ private final AggregationOperation aggOp;
+
+ /**
+ * Private constructor, meant to be used by the enum definition.
+ *
+ * @param columnFamily
+ * that this column is stored in.
+ * @param columnPrefix
+ * for this column.
+ */
+ private FlowActivityColumnPrefix(
+ ColumnFamily columnFamily, String columnPrefix,
+ AggregationOperation aggOp) {
+ this(columnFamily, columnPrefix, aggOp, false);
+ }
+
+ private FlowActivityColumnPrefix(
+ ColumnFamily columnFamily, String columnPrefix,
+ AggregationOperation aggOp, boolean compoundColQual) {
+ this.columnFamily = columnFamily;
+ this.columnPrefix = columnPrefix;
+ if (columnPrefix == null) {
+ this.columnPrefixBytes = null;
+ } else {
+ // Future-proof by ensuring the right column prefix hygiene.
+ this.columnPrefixBytes = Bytes.toBytes(Separator.SPACE
+ .encode(columnPrefix));
+ }
+ this.converter = GenericConverter.getInstance();
+ this.aggOp = aggOp;
+ }
+
+ /**
+ * @return the column name value
+ */
+ public String getColumnPrefix() {
+ return columnPrefix;
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(
+ this.columnPrefixBytes, qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(String qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(
+ this.columnPrefixBytes, qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes() {
+ return columnPrefixBytes != null ? columnPrefixBytes.clone() : null;
+ }
+
+ @Override
+ public byte[] getColumnFamilyBytes() {
+ return columnFamily.getBytes();
+ }
+
+ @Override
+ public ValueConverter getValueConverter() {
+ return converter;
+ }
+
+ public AggregationOperation getAttribute() {
+ return aggOp;
+ }
+
+ @Override
+ public Attribute[] combineAttrsWithAggr(Attribute... attributes) {
+ return HBaseTimelineSchemaUtils.combineAttributes(attributes, aggOp);
+ }
+
+ /**
+ * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there
+ * is no match. The following holds true:
+ * {@code columnFor(a,x) == columnFor(b,y)} if and only if
+ * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
+ *
+ * @param columnFamily
+ * The columnFamily for which to retrieve the column.
+ * @param columnPrefix
+ * Name of the column to retrieve
+ * @return the corresponding {@link FlowActivityColumnPrefix} or null if both
+ * arguments don't match.
+ */
+ public static final FlowActivityColumnPrefix columnFor(
+ FlowActivityColumnFamily columnFamily, String columnPrefix) {
+
+ // TODO: needs unit test to confirm and need to update javadoc to explain
+ // null prefix case.
+
+ for (FlowActivityColumnPrefix flowActivityColumnPrefix :
+ FlowActivityColumnPrefix.values()) {
+ // Find a match based column family and on name.
+ if (flowActivityColumnPrefix.columnFamily.equals(columnFamily)
+ && (((columnPrefix == null) && (flowActivityColumnPrefix
+ .getColumnPrefix() == null)) || (flowActivityColumnPrefix
+ .getColumnPrefix().equals(columnPrefix)))) {
+ return flowActivityColumnPrefix;
+ }
+ }
+ // Default to null
+ return null;
+ }
+}
\ No newline at end of file
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
similarity index 98%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
index bb77e36..9f2044a 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
@@ -18,7 +18,7 @@
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.HBaseTimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
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;
@@ -59,7 +59,7 @@ protected FlowActivityRowKey(String clusterId, Long timestamp, String userId,
String flowName, boolean convertDayTsToTopOfDay) {
this.clusterId = clusterId;
if (convertDayTsToTopOfDay && (timestamp != null)) {
- this.dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(timestamp);
+ this.dayTs = HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(timestamp);
} else {
this.dayTs = timestamp;
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
new file mode 100644
index 0000000..5688cc5
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
@@ -0,0 +1,6 @@
+package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
+
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+
+public class FlowActivityTable implements BaseTable {
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
new file mode 100644
index 0000000..856790e
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
@@ -0,0 +1,159 @@
+/**
+ * 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.Column;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
+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.ValueConverter;
+
+/**
+ * Identifies fully qualified columns for the {@link FlowRunTable}.
+ */
+public enum FlowRunColumn implements Column {
+
+ /**
+ * When the flow was started. This is the minimum of currently known
+ * application start times.
+ */
+ MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time",
+ 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, new LongConverter()),
+
+ /**
+ * The version of the flow that this flow belongs to.
+ */
+ FLOW_VERSION(FlowRunColumnFamily.INFO, "flow_version", null);
+
+ private final ColumnFamily columnFamily;
+ private final String columnQualifier;
+ private final byte[] columnQualifierBytes;
+ private final ValueConverter converter;
+ private final AggregationOperation aggOp;
+
+ private FlowRunColumn(ColumnFamily columnFamily,
+ String columnQualifier, AggregationOperation aggOp) {
+ this(columnFamily, columnQualifier, aggOp,
+ GenericConverter.getInstance());
+ }
+
+ private FlowRunColumn(ColumnFamily columnFamily,
+ String columnQualifier, AggregationOperation aggOp,
+ ValueConverter converter) {
+ this.columnFamily = columnFamily;
+ this.columnQualifier = columnQualifier;
+ this.aggOp = aggOp;
+ // Future-proof by ensuring the right column prefix hygiene.
+ this.columnQualifierBytes = Bytes.toBytes(Separator.SPACE
+ .encode(columnQualifier));
+ this.converter = converter;
+ }
+
+ /**
+ * @return the column name value
+ */
+ private String getColumnQualifier() {
+ return columnQualifier;
+ }
+
+ @Override
+ public byte[] getColumnQualifierBytes() {
+ return columnQualifierBytes.clone();
+ }
+
+ @Override
+ public byte[] getColumnFamilyBytes() {
+ return columnFamily.getBytes();
+ }
+
+ @Override
+ public Attribute[] combineAttrsWithAggr(Attribute... attributes) {
+ return HBaseTimelineSchemaUtils.combineAttributes(attributes, aggOp);
+ }
+
+ public AggregationOperation getAggregationOperation() {
+ return aggOp;
+ }
+
+ /**
+ * Retrieve an {@link FlowRunColumn} given a name, or null if there is no
+ * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
+ * and only if {@code x.equals(y)} or {@code (x == y == null)}
+ *
+ * @param columnQualifier
+ * Name of the column to retrieve
+ * @return the corresponding {@link FlowRunColumn} or null
+ */
+ public static final FlowRunColumn columnFor(String columnQualifier) {
+
+ // Match column based on value, assume column family matches.
+ for (FlowRunColumn ec : FlowRunColumn.values()) {
+ // Find a match based only on name.
+ if (ec.getColumnQualifier().equals(columnQualifier)) {
+ return ec;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+ @Override
+ public ValueConverter getValueConverter() {
+ return converter;
+ }
+
+ /**
+ * Retrieve an {@link FlowRunColumn} given a name, or null if there is no
+ * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
+ * if and only if {@code a.equals(b) & x.equals(y)} or
+ * {@code (x == y == null)}
+ *
+ * @param columnFamily
+ * The columnFamily for which to retrieve the column.
+ * @param name
+ * Name of the column to retrieve
+ * @return the corresponding {@link FlowRunColumn} or null if both arguments
+ * don't match.
+ */
+ public static final FlowRunColumn columnFor(FlowRunColumnFamily columnFamily,
+ String name) {
+
+ for (FlowRunColumn ec : FlowRunColumn.values()) {
+ // Find a match based column family and on name.
+ if (ec.columnFamily.equals(columnFamily)
+ && ec.getColumnQualifier().equals(name)) {
+ return ec;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
new file mode 100644
index 0000000..8fc96a0
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
@@ -0,0 +1,172 @@
+/**
+ * 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.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
+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.ValueConverter;
+
+/**
+ * Identifies partially qualified columns for the {@link FlowRunTable}.
+ */
+public enum FlowRunColumnPrefix implements ColumnPrefix {
+
+ /**
+ * To store flow run info values.
+ */
+ METRIC(FlowRunColumnFamily.INFO, "m", null, new LongConverter());
+
+ private final ColumnFamily columnFamily;
+
+ /**
+ * Can be null for those cases where the provided column qualifier is the
+ * entire column name.
+ */
+ private final String columnPrefix;
+ private final byte[] columnPrefixBytes;
+ private final ValueConverter converter;
+
+ private final AggregationOperation aggOp;
+
+ /**
+ * Private constructor, meant to be used by the enum definition.
+ *
+ * @param columnFamily that this column is stored in.
+ * @param columnPrefix for this column.
+ */
+ private FlowRunColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix, AggregationOperation fra, ValueConverter converter) {
+ this(columnFamily, columnPrefix, fra, converter, false);
+ }
+
+ private FlowRunColumnPrefix(ColumnFamily columnFamily,
+ String columnPrefix, AggregationOperation fra, ValueConverter converter,
+ boolean compoundColQual) {
+ this.columnFamily = columnFamily;
+ this.columnPrefix = columnPrefix;
+ if (columnPrefix == null) {
+ this.columnPrefixBytes = null;
+ } else {
+ // Future-proof by ensuring the right column prefix hygiene.
+ this.columnPrefixBytes =
+ Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
+ }
+ this.converter = converter;
+ this.aggOp = fra;
+ }
+
+ /**
+ * @return the column name value
+ */
+ public String getColumnPrefix() {
+ return columnPrefix;
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes() {
+ return columnPrefixBytes != null ? columnPrefixBytes.clone() : null;
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(this.columnPrefixBytes,
+ qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnPrefixBytes(String qualifierPrefix) {
+ return ColumnHelper.getColumnQualifier(this.columnPrefixBytes,
+ qualifierPrefix);
+ }
+
+ @Override
+ public byte[] getColumnFamilyBytes() {
+ return columnFamily.getBytes();
+ }
+
+ public AggregationOperation getAttribute() {
+ return aggOp;
+ }
+
+ @Override
+ public Attribute[] combineAttrsWithAggr(Attribute... attributes) {
+ return HBaseTimelineSchemaUtils.combineAttributes(attributes, aggOp);
+ }
+
+ /**
+ * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is
+ * no match. The following holds true: {@code columnFor(x) == columnFor(y)} if
+ * and only if {@code x.equals(y)} or {@code (x == y == null)}
+ *
+ * @param columnPrefix Name of the column to retrieve
+ * @return the corresponding {@link FlowRunColumnPrefix} or null
+ */
+ public static final FlowRunColumnPrefix columnFor(String columnPrefix) {
+
+ // Match column based on value, assume column family matches.
+ for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) {
+ // Find a match based only on name.
+ if (frcp.getColumnPrefix().equals(columnPrefix)) {
+ return frcp;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+
+ @Override
+ public ValueConverter getValueConverter() {
+ return converter;
+ }
+
+ /**
+ * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is
+ * no match. The following holds true:
+ * {@code columnFor(a,x) == columnFor(b,y)} if and only if
+ * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
+ *
+ * @param columnFamily The columnFamily for which to retrieve the column.
+ * @param columnPrefix Name of the column to retrieve
+ * @return the corresponding {@link FlowRunColumnPrefix} or null if both
+ * arguments don't match.
+ */
+ public static final FlowRunColumnPrefix columnFor(
+ FlowRunColumnFamily columnFamily, String columnPrefix) {
+
+ // TODO: needs unit test to confirm and need to update javadoc to explain
+ // null prefix case.
+
+ for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) {
+ // Find a match based column family and on name.
+ if (frcp.columnFamily.equals(columnFamily)
+ && (((columnPrefix == null) && (frcp.getColumnPrefix() == null)) ||
+ (frcp.getColumnPrefix().equals(columnPrefix)))) {
+ return frcp;
+ }
+ }
+
+ // Default to null
+ return null;
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
new file mode 100644
index 0000000..4e88623
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
@@ -0,0 +1,6 @@
+package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
+
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+
+public class FlowRunTable implements BaseTable {
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java
new file mode 100644
index 0000000..04963f3
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * 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
+ * contains classes related to implementation for flow related tables, viz. flow
+ * run table and flow activity table.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
similarity index 99%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
index cbd2273..c1c8bd7 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
@@ -189,7 +189,7 @@ public void testEntityRowKey() {
@Test
public void testFlowActivityRowKey() {
Long ts = 1459900830000L;
- Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts);
+ Long dayTimestamp = HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(ts);
byte[] byteRowKey =
new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME).getRowKey();
FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(byteRowKey);
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java
similarity index 100%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-schema/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/pom.xml
new file mode 100644
index 0000000..66c2098
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/pom.xml
@@ -0,0 +1,199 @@
+
+
+
+
+ hadoop-yarn-server-timelineservice-hbase
+ org.apache.hadoop
+ 3.0.0-alpha2-SNAPSHOT
+
+ 4.0.0
+
+ hadoop-yarn-server-timelineservice-hbase-server
+
+
+
+ ${project.parent.parent.parent.basedir}
+
+
+
+
+ commons-logging
+ commons-logging
+
+
+
+ com.google.guava
+ guava
+
+
+
+ org.apache.hadoop
+ hadoop-annotations
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ ${hbase-compatible-hadoop.version}
+
+
+ org.apache.hadoop
+ hadoop-auth
+
+
+ tomcat
+ jasper-runtime
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-api
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-timelineservice-hbase-schema
+
+
+ hadoop-common
+ org.apache.hadoop
+
+
+
+
+
+ org.apache.hbase
+ hbase-common
+
+
+ hadoop-common
+ org.apache.hadoop
+
+
+ org.apache.hadoop
+ hadoop-mapreduce-client-core
+
+
+ org.mortbay.jetty
+ jetty-util
+
+
+
+
+
+ org.apache.hbase
+ hbase-client
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+ org.apache.hadoop
+ hadoop-auth
+
+
+ org.apache.hadoop
+ hadoop-mapreduce-client-core
+
+
+
+
+
+ org.apache.hbase
+ hbase-server
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+ org.apache.hadoop
+ hadoop-auth
+
+
+ org.apache.hadoop
+ hadoop-hdfs
+
+
+ org.apache.hadoop
+ hadoop-hdfs-client
+
+
+ org.apache.hadoop
+ hadoop-client
+
+
+ org.apache.hadoop
+ hadoop-mapreduce-client-core
+
+
+ org.mortbay.jetty
+ jetty
+
+
+ org.mortbay.jetty
+ jetty-util
+
+
+ org.mortbay.jetty
+ jetty-sslengine
+
+
+
+
+
+
+
+
+ maven-jar-plugin
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+
+ org.apache.maven.plugins
+ maven-javadoc-plugin
+
+
+
+ junit
+ junit
+ 4.11
+
+
+
+
+
+
+
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
similarity index 98%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
index 24101c6..1b4a9a5 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
@@ -48,7 +48,7 @@
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
/**
@@ -96,7 +96,7 @@ public void prePut(ObserverContext e, Put put,
List tags = new ArrayList<>();
if ((attributes != null) && (attributes.size() > 0)) {
for (Map.Entry attribute : attributes.entrySet()) {
- Tag t = HBaseTimelineStorageUtils.getTagFromAttribute(attribute);
+ Tag t = HBaseTimelineSchemaUtils.getTagFromAttribute(attribute);
tags.add(t);
}
byte[] tagByteArray = Tag.fromList(tags);
@@ -149,7 +149,7 @@ private long getCellTimestamp(long timestamp, List tags) {
* (non-Javadoc)
*
* Creates a {@link FlowScanner} Scan so that it can correctly process the
- * contents of {@link FlowRunTable}.
+ * contents of {@link FlowRunTableRW}.
*
* @see
* org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preGetOp(org.apache
@@ -200,7 +200,7 @@ public RegionScanner preScannerOpen(
* (non-Javadoc)
*
* Creates a {@link FlowScanner} Scan so that it can correctly process the
- * contents of {@link FlowRunTable}.
+ * contents of {@link FlowRunTableRW}.
*
* @see
* org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#postScannerOpen(
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
similarity index 98%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
index 0e3c8ee..c9d6ac9 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
@@ -45,7 +45,7 @@
import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineSchemaUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.NumericValueConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
@@ -249,7 +249,7 @@ private AggregationOperation getCurrentAggOp(Cell cell) {
List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
cell.getTagsLength());
// We assume that all the operations for a particular column are the same
- return HBaseTimelineStorageUtils.getAggregationOperationFromTagsList(tags);
+ return HBaseTimelineSchemaUtils.getAggregationOperationFromTagsList(tags);
}
/**
@@ -323,7 +323,7 @@ private void collectCells(SortedSet currentColumnCells,
// only if this app has not been seen yet, add to current column cells
List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
cell.getTagsLength());
- String aggDim = HBaseTimelineStorageUtils
+ String aggDim = HBaseTimelineSchemaUtils
.getAggregationCompactionDimension(tags);
if (!alreadySeenAggDim.contains(aggDim)) {
// if this agg dimension has already been seen,
@@ -419,7 +419,7 @@ private Cell processSummation(SortedSet currentColumnCells,
}
byte[] sumBytes = converter.encodeValue(sum);
Cell sumCell =
- HBaseTimelineStorageUtils.createNewCell(mostRecentCell, sumBytes);
+ HBaseTimelineSchemaUtils.createNewCell(mostRecentCell, sumBytes);
return sumCell;
}
@@ -461,7 +461,7 @@ private Cell processSummation(SortedSet currentColumnCells,
// if this is the existing flow sum cell
List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
cell.getTagsLength());
- String appId = HBaseTimelineStorageUtils
+ String appId = HBaseTimelineSchemaUtils
.getAggregationCompactionDimension(tags);
if (appId == FLOW_APP_ID) {
sum = converter.add(sum, currentValue);
@@ -503,7 +503,7 @@ private Cell processSummation(SortedSet| currentColumnCells,
Bytes.toBytes(FLOW_APP_ID));
tags.add(t);
byte[] tagByteArray = Tag.fromList(tags);
- Cell sumCell = HBaseTimelineStorageUtils.createNewCell(
+ Cell sumCell = HBaseTimelineSchemaUtils.createNewCell(
CellUtil.cloneRow(anyCell),
CellUtil.cloneFamily(anyCell),
CellUtil.cloneQualifier(anyCell),
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java
similarity index 94%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java
index 73c666f..fac32f5 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java
@@ -18,6 +18,8 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+
/**
* Identifies the scanner operation on the {@link FlowRunTable}.
*/
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java
new file mode 100644
index 0000000..04963f3
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * 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
+ * contains classes related to implementation for flow related tables, viz. flow
+ * run table and flow activity table.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
index a2ba4e8..a788884 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
@@ -25,166 +25,16 @@
3.0.0-alpha2-SNAPSHOT
| | | | | | | | | | | |
4.0.0
+ org.apache.hadoop
hadoop-yarn-server-timelineservice-hbase
+ 3.0.0-alpha2-SNAPSHOT
Apache Hadoop YARN TimelineService HBase Backend
+ pom
-
-
- ${project.parent.parent.basedir}
-
+
+ hadoop-yarn-server-timelineservice-hbase-client
+ hadoop-yarn-server-timelineservice-hbase-server
+ hadoop-yarn-server-timelineservice-hbase-schema
+
-
-
- commons-logging
- commons-logging
-
-
-
- commons-lang
- commons-lang
-
-
-
- commons-cli
- commons-cli
-
-
-
- com.google.guava
- guava
-
-
-
- org.apache.hadoop
- hadoop-annotations
-
-
-
- org.apache.hadoop
- hadoop-common
-
-
-
- org.apache.hadoop
- hadoop-common
- test-jar
- test
-
-
-
- org.apache.hadoop
- hadoop-yarn-api
-
-
-
- org.apache.hadoop
- hadoop-yarn-common
-
-
-
- org.apache.hadoop
- hadoop-yarn-server-applicationhistoryservice
-
-
-
- org.apache.hadoop
- hadoop-yarn-server-timelineservice
-
-
-
- org.apache.hbase
- hbase-common
-
-
- org.apache.hadoop
- hadoop-mapreduce-client-core
-
-
- org.mortbay.jetty
- jetty-util
-
-
-
-
-
- org.apache.hbase
- hbase-client
-
-
- org.apache.hadoop
- hadoop-mapreduce-client-core
-
-
-
-
-
- org.apache.hbase
- hbase-server
-
-
- org.apache.hadoop
- hadoop-hdfs
-
-
- org.apache.hadoop
- hadoop-hdfs-client
-
-
- org.apache.hadoop
- hadoop-client
-
-
- org.apache.hadoop
- hadoop-mapreduce-client-core
-
-
- org.mortbay.jetty
- jetty
-
-
- org.mortbay.jetty
- jetty-util
-
-
- org.mortbay.jetty
- jetty-sslengine
-
-
-
-
-
- junit
- junit
- test
-
-
-
-
-
-
- maven-jar-plugin
-
-
-
- test-jar
-
- test-compile
-
-
-
-
- org.apache.maven.plugins
- maven-javadoc-plugin
-
-
-
- junit
- junit
- 4.11
-
-
-
-
-
-
-
\ No newline at end of file
+
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
deleted file mode 100644
index 642df63..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
+++ /dev/null
@@ -1,547 +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;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.service.AbstractService;
-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;
-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;
-import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
-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.apptoflow.AppToFlowColumnPrefix;
-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.ColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
-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.TypedBufferedMutator;
-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.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;
-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.FlowActivityTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
-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.FlowRunTable;
-
-/**
- * This implements a hbase based backend for storing the timeline entity
- * information.
- * It writes to multiple tables at the backend
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class HBaseTimelineWriterImpl extends AbstractService implements
- TimelineWriter {
-
- private static final Log LOG = LogFactory
- .getLog(HBaseTimelineWriterImpl.class);
-
- private Connection conn;
- private TypedBufferedMutator entityTable;
- private TypedBufferedMutator appToFlowTable;
- private TypedBufferedMutator applicationTable;
- private TypedBufferedMutator flowActivityTable;
- private TypedBufferedMutator flowRunTable;
-
- /**
- * Used to convert strings key components to and from storage format.
- */
- private final KeyConverter stringKeyConverter =
- new StringKeyConverter();
-
- /**
- * Used to convert Long key components to and from storage format.
- */
- private final KeyConverter longKeyConverter = new LongKeyConverter();
-
- public HBaseTimelineWriterImpl() {
- super(HBaseTimelineWriterImpl.class.getName());
- }
-
- public HBaseTimelineWriterImpl(Configuration conf) throws IOException {
- super(conf.get("yarn.application.id",
- HBaseTimelineWriterImpl.class.getName()));
- }
-
- /**
- * initializes the hbase connection to write to the entity table.
- */
- @Override
- protected void serviceInit(Configuration conf) throws Exception {
- super.serviceInit(conf);
- Configuration hbaseConf =
- HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(conf);
- conn = ConnectionFactory.createConnection(hbaseConf);
- entityTable = new EntityTable().getTableMutator(hbaseConf, conn);
- appToFlowTable = new AppToFlowTable().getTableMutator(hbaseConf, conn);
- applicationTable = new ApplicationTable().getTableMutator(hbaseConf, conn);
- flowRunTable = new FlowRunTable().getTableMutator(hbaseConf, conn);
- flowActivityTable =
- new FlowActivityTable().getTableMutator(hbaseConf, conn);
- }
-
- /**
- * Stores the entire information in TimelineEntities to the timeline store.
- */
- @Override
- public TimelineWriteResponse write(String clusterId, String userId,
- String flowName, String flowVersion, long flowRunId, String appId,
- TimelineEntities data) throws IOException {
-
- TimelineWriteResponse putStatus = new TimelineWriteResponse();
- // defensive coding to avoid NPE during row key construction
- if ((flowName == null) || (appId == null) || (clusterId == null)
- || (userId == null)) {
- LOG.warn("Found null for one of: flowName=" + flowName + " appId=" + appId
- + " userId=" + userId + " clusterId=" + clusterId
- + " . Not proceeding with writing to hbase");
- return putStatus;
- }
-
- for (TimelineEntity te : data.getEntities()) {
-
- // a set can have at most 1 null
- if (te == null) {
- continue;
- }
-
- // if the entity is the application, the destination is the application
- // table
- boolean isApplication = ApplicationEntity.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.getIdPrefix(), te.getId());
- rowKey = entityRowKey.getRowKey();
- }
-
- storeInfo(rowKey, te, flowVersion, isApplication);
- storeEvents(rowKey, te.getEvents(), isApplication);
- storeConfig(rowKey, te.getConfigs(), isApplication);
- storeMetrics(rowKey, te.getMetrics(), isApplication);
- storeRelations(rowKey, te, isApplication);
-
- if (isApplication) {
- TimelineEvent event =
- ApplicationEntity.getApplicationEvent(te,
- ApplicationMetricsConstants.CREATED_EVENT_TYPE);
- FlowRunRowKey flowRunRowKey =
- new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
- if (event != null) {
- onApplicationCreated(flowRunRowKey, clusterId, appId, userId,
- flowVersion, te, event.getTimestamp());
- }
- // if it's an application entity, store metrics
- storeFlowMetricsAppRunning(flowRunRowKey, appId, te);
- // if application has finished, store it's finish time and write final
- // values of all metrics
- event = ApplicationEntity.getApplicationEvent(te,
- ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
- if (event != null) {
- onApplicationFinished(flowRunRowKey, flowVersion, appId, te,
- event.getTimestamp());
- }
- }
- }
- return putStatus;
- }
-
- private void onApplicationCreated(FlowRunRowKey flowRunRowKey,
- String clusterId, String appId, String userId, String flowVersion,
- TimelineEntity te, long appCreatedTimeStamp)
- throws IOException {
-
- String flowName = flowRunRowKey.getFlowName();
- Long flowRunId = flowRunRowKey.getFlowRunId();
-
- // store in App to flow table
- AppToFlowRowKey appToFlowRowKey = new AppToFlowRowKey(appId);
- byte[] rowKey = appToFlowRowKey.getRowKey();
- AppToFlowColumnPrefix.FLOW_NAME.store(rowKey, appToFlowTable, clusterId,
- null, flowName);
- AppToFlowColumnPrefix.FLOW_RUN_ID.store(rowKey, appToFlowTable, clusterId,
- null, flowRunId);
- AppToFlowColumnPrefix.USER_ID.store(rowKey, appToFlowTable, clusterId, 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(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));
- }
-
-
- /*
- * updates the {@link FlowRunTable} and {@link FlowActivityTable} when an
- * application has finished
- */
- private void onApplicationFinished(FlowRunRowKey flowRunRowKey,
- String flowVersion, String appId, TimelineEntity te,
- long appFinishedTimeStamp) throws IOException {
- // store in flow run table
- storeAppFinishedInFlowRunTable(flowRunRowKey, appId, te,
- appFinishedTimeStamp);
-
- // indicate in the flow activity table that the app has finished
- 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(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,
- appFinishedTimeStamp, attributeAppId);
-
- // store the final value of metrics since application has finished
- Set metrics = te.getMetrics();
- if (metrics != null) {
- storeFlowMetrics(rowKey, metrics, attributeAppId,
- AggregationOperation.SUM_FINAL.getAttribute());
- }
- }
-
- /*
- * Updates the {@link FlowRunTable} with Application Metrics
- */
- private void storeFlowMetricsAppRunning(FlowRunRowKey flowRunRowKey,
- String appId, TimelineEntity te) throws IOException {
- Set metrics = te.getMetrics();
- if (metrics != null) {
- byte[] rowKey = flowRunRowKey.getRowKey();
- storeFlowMetrics(rowKey, metrics,
- AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId),
- AggregationOperation.SUM.getAttribute());
- }
- }
-
- private void storeFlowMetrics(byte[] rowKey, Set metrics,
- Attribute... attributes) throws IOException {
- for (TimelineMetric metric : metrics) {
- byte[] metricColumnQualifier = stringKeyConverter.encode(metric.getId());
- Map timeseries = metric.getValues();
- for (Map.Entry timeseriesEntry : timeseries.entrySet()) {
- Long timestamp = timeseriesEntry.getKey();
- FlowRunColumnPrefix.METRIC.store(rowKey, flowRunTable,
- metricColumnQualifier, timestamp, timeseriesEntry.getValue(),
- attributes);
- }
- }
- }
-
- private void storeRelations(byte[] rowKey, TimelineEntity te,
- boolean isApplication) throws IOException {
- if (isApplication) {
- storeRelations(rowKey, te.getIsRelatedToEntities(),
- ApplicationColumnPrefix.IS_RELATED_TO, applicationTable);
- storeRelations(rowKey, te.getRelatesToEntities(),
- ApplicationColumnPrefix.RELATES_TO, applicationTable);
- } else {
- storeRelations(rowKey, te.getIsRelatedToEntities(),
- EntityColumnPrefix.IS_RELATED_TO, entityTable);
- storeRelations(rowKey, te.getRelatesToEntities(),
- EntityColumnPrefix.RELATES_TO, entityTable);
- }
- }
-
- /**
- * Stores the Relations from the {@linkplain TimelineEntity} object.
- */
- private void storeRelations(byte[] rowKey,
- Map> connectedEntities,
- ColumnPrefix columnPrefix, TypedBufferedMutator table)
- throws IOException {
- for (Map.Entry> connectedEntity : connectedEntities
- .entrySet()) {
- // id3?id4?id5
- String compoundValue =
- Separator.VALUES.joinEncoded(connectedEntity.getValue());
- columnPrefix.store(rowKey, table,
- stringKeyConverter.encode(connectedEntity.getKey()), null,
- compoundValue);
- }
- }
-
- /**
- * Stores information from the {@linkplain TimelineEntity} object.
- */
- private void storeInfo(byte[] rowKey, TimelineEntity te, String flowVersion,
- boolean isApplication) throws IOException {
-
- if (isApplication) {
- ApplicationColumn.ID.store(rowKey, applicationTable, null, te.getId());
- ApplicationColumn.CREATED_TIME.store(rowKey, applicationTable, null,
- te.getCreatedTime());
- ApplicationColumn.FLOW_VERSION.store(rowKey, applicationTable, null,
- flowVersion);
- Map info = te.getInfo();
- if (info != null) {
- for (Map.Entry entry : info.entrySet()) {
- ApplicationColumnPrefix.INFO.store(rowKey, applicationTable,
- stringKeyConverter.encode(entry.getKey()), null,
- entry.getValue());
- }
- }
- } else {
- EntityColumn.ID.store(rowKey, entityTable, null, te.getId());
- EntityColumn.TYPE.store(rowKey, entityTable, null, te.getType());
- EntityColumn.CREATED_TIME.store(rowKey, entityTable, null,
- te.getCreatedTime());
- EntityColumn.FLOW_VERSION.store(rowKey, entityTable, null, flowVersion);
- Map info = te.getInfo();
- if (info != null) {
- for (Map.Entry entry : info.entrySet()) {
- EntityColumnPrefix.INFO.store(rowKey, entityTable,
- stringKeyConverter.encode(entry.getKey()), null,
- entry.getValue());
- }
- }
- }
- }
-
- /**
- * stores the config information from {@linkplain TimelineEntity}.
- */
- private void storeConfig(byte[] rowKey, Map config,
- boolean isApplication) throws IOException {
- if (config == null) {
- return;
- }
- for (Map.Entry entry : config.entrySet()) {
- byte[] configKey = stringKeyConverter.encode(entry.getKey());
- if (isApplication) {
- ApplicationColumnPrefix.CONFIG.store(rowKey, applicationTable,
- configKey, null, entry.getValue());
- } else {
- EntityColumnPrefix.CONFIG.store(rowKey, entityTable, configKey,
- null, entry.getValue());
- }
- }
- }
-
- /**
- * stores the {@linkplain TimelineMetric} information from the
- * {@linkplain TimelineEvent} object.
- */
- private void storeMetrics(byte[] rowKey, Set metrics,
- boolean isApplication) throws IOException {
- if (metrics != null) {
- for (TimelineMetric metric : metrics) {
- byte[] metricColumnQualifier =
- stringKeyConverter.encode(metric.getId());
- Map timeseries = metric.getValues();
- for (Map.Entry timeseriesEntry : timeseries.entrySet()) {
- Long timestamp = timeseriesEntry.getKey();
- if (isApplication) {
- ApplicationColumnPrefix.METRIC.store(rowKey, applicationTable,
- metricColumnQualifier, timestamp, timeseriesEntry.getValue());
- } else {
- EntityColumnPrefix.METRIC.store(rowKey, entityTable,
- metricColumnQualifier, timestamp, timeseriesEntry.getValue());
- }
- }
- }
- }
- }
-
- /**
- * Stores the events from the {@linkplain TimelineEvent} object.
- */
- private void storeEvents(byte[] rowKey, Set events,
- boolean isApplication) throws IOException {
- if (events != null) {
- for (TimelineEvent event : events) {
- if (event != null) {
- String eventId = event.getId();
- if (eventId != null) {
- long eventTimestamp = event.getTimestamp();
- // if the timestamp is not set, use the current timestamp
- if (eventTimestamp == TimelineEvent.INVALID_TIMESTAMP) {
- LOG.warn("timestamp is not set for event " + eventId +
- "! Using the current timestamp");
- eventTimestamp = System.currentTimeMillis();
- }
- Map eventInfo = event.getInfo();
- if ((eventInfo == null) || (eventInfo.size() == 0)) {
- byte[] columnQualifierBytes =
- new EventColumnName(eventId, eventTimestamp, null)
- .getColumnQualifier();
- if (isApplication) {
- ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
- columnQualifierBytes, null, Separator.EMPTY_BYTES);
- } else {
- EntityColumnPrefix.EVENT.store(rowKey, entityTable,
- columnQualifierBytes, null, Separator.EMPTY_BYTES);
- }
- } else {
- for (Map.Entry info : eventInfo.entrySet()) {
- // eventId=infoKey
- byte[] columnQualifierBytes =
- new EventColumnName(eventId, eventTimestamp, info.getKey())
- .getColumnQualifier();
- if (isApplication) {
- ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
- columnQualifierBytes, null, info.getValue());
- } else {
- EntityColumnPrefix.EVENT.store(rowKey, entityTable,
- columnQualifierBytes, null, info.getValue());
- }
- } // for info: eventInfo
- }
- }
- }
- } // event : events
- }
- }
-
- /*
- * (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
- entityTable.flush();
- appToFlowTable.flush();
- applicationTable.flush();
- flowRunTable.flush();
- flowActivityTable.flush();
- }
-
- /**
- * close the hbase connections The close APIs perform flushing and release any
- * resources held.
- */
- @Override
- protected void serviceStop() throws Exception {
- if (entityTable != null) {
- LOG.info("closing the entity table");
- // The close API performs flushing and releases any resources held
- entityTable.close();
- }
- if (appToFlowTable != null) {
- LOG.info("closing the app_flow table");
- // The close API performs flushing and releases any resources held
- appToFlowTable.close();
- }
- if (applicationTable != null) {
- LOG.info("closing the application table");
- applicationTable.close();
- }
- if (flowRunTable != null) {
- LOG.info("closing the flow run table");
- // The close API performs flushing and releases any resources held
- flowRunTable.close();
- }
- if (flowActivityTable != null) {
- LOG.info("closing the flowActivityTable table");
- // The close API performs flushing and releases any resources held
- flowActivityTable.close();
- }
- if (conn != null) {
- LOG.info("closing the hbase Connection");
- conn.close();
- }
- super.serviceStop();
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java
deleted file mode 100644
index dde3911..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java
+++ /dev/null
@@ -1,156 +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 java.io.IOException;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-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.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * Identifies fully qualified columns for the {@link ApplicationTable}.
- */
-public enum ApplicationColumn implements Column {
-
- /**
- * App id.
- */
- ID(ApplicationColumnFamily.INFO, "id"),
-
- /**
- * When the application was created.
- */
- CREATED_TIME(ApplicationColumnFamily.INFO, "created_time",
- new LongConverter()),
-
- /**
- * The version of the flow that this app belongs to.
- */
- FLOW_VERSION(ApplicationColumnFamily.INFO, "flow_version");
-
- private final ColumnHelper column;
- private final ColumnFamily columnFamily;
- private final String columnQualifier;
- private final byte[] columnQualifierBytes;
-
- private ApplicationColumn(ColumnFamily columnFamily,
- String columnQualifier) {
- this(columnFamily, columnQualifier, GenericConverter.getInstance());
- }
-
- private ApplicationColumn(ColumnFamily columnFamily,
- String columnQualifier, ValueConverter converter) {
- this.columnFamily = columnFamily;
- this.columnQualifier = columnQualifier;
- // Future-proof by ensuring the right column prefix hygiene.
- this.columnQualifierBytes =
- Bytes.toBytes(Separator.SPACE.encode(columnQualifier));
- this.column = new ColumnHelper(columnFamily, converter);
- }
-
- /**
- * @return the column name value
- */
- private String getColumnQualifier() {
- return columnQualifier;
- }
-
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, Long timestamp,
- Object inputValue, Attribute... attributes) throws IOException {
- column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
- inputValue, attributes);
- }
-
- public Object readResult(Result result) throws IOException {
- return column.readResult(result, columnQualifierBytes);
- }
-
- @Override
- public byte[] getColumnQualifierBytes() {
- return columnQualifierBytes.clone();
- }
-
- @Override
- public byte[] getColumnFamilyBytes() {
- return columnFamily.getBytes();
- }
-
- @Override
- public ValueConverter getValueConverter() {
- return column.getValueConverter();
- }
-
- /**
- * Retrieve an {@link ApplicationColumn} given a name, or null if there is no
- * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
- * and only if {@code x.equals(y)} or {@code (x == y == null)}.
- *
- * @param columnQualifier Name of the column to retrieve
- * @return the corresponding {@link ApplicationColumn} or null
- */
- public static final ApplicationColumn columnFor(String columnQualifier) {
-
- // Match column based on value, assume column family matches.
- for (ApplicationColumn ac : ApplicationColumn.values()) {
- // Find a match based only on name.
- if (ac.getColumnQualifier().equals(columnQualifier)) {
- return ac;
- }
- }
-
- // Default to null
- return null;
- }
-
- /**
- * Retrieve an {@link ApplicationColumn} given a name, or null if there is no
- * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
- * if and only if {@code a.equals(b) & x.equals(y)} or
- * {@code (x == y == null)}
- *
- * @param columnFamily The columnFamily for which to retrieve the column.
- * @param name Name of the column to retrieve
- * @return the corresponding {@link ApplicationColumn} or null if both
- * arguments don't match.
- */
- public static final ApplicationColumn columnFor(
- ApplicationColumnFamily columnFamily, String name) {
-
- for (ApplicationColumn ac : ApplicationColumn.values()) {
- // Find a match based column family and on name.
- if (ac.columnFamily.equals(columnFamily)
- && ac.getColumnQualifier().equals(name)) {
- return ac;
- }
- }
-
- // Default to null
- return null;
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
deleted file mode 100644
index 42488f4..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
+++ /dev/null
@@ -1,288 +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 java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-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.ColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
-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.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * Identifies partially qualified columns for the application table.
- */
-public enum ApplicationColumnPrefix implements ColumnPrefix {
-
- /**
- * To store TimelineEntity getIsRelatedToEntities values.
- */
- IS_RELATED_TO(ApplicationColumnFamily.INFO, "s"),
-
- /**
- * To store TimelineEntity getRelatesToEntities values.
- */
- RELATES_TO(ApplicationColumnFamily.INFO, "r"),
-
- /**
- * To store TimelineEntity info values.
- */
- INFO(ApplicationColumnFamily.INFO, "i"),
-
- /**
- * Lifecycle events for an application.
- */
- EVENT(ApplicationColumnFamily.INFO, "e"),
-
- /**
- * Config column stores configuration with config key as the column name.
- */
- CONFIG(ApplicationColumnFamily.CONFIGS, null),
-
- /**
- * Metrics are stored with the metric name as the column name.
- */
- METRIC(ApplicationColumnFamily.METRICS, null, new LongConverter());
-
- private final ColumnHelper column;
- private final ColumnFamily columnFamily;
-
- /**
- * Can be null for those cases where the provided column qualifier is the
- * entire column name.
- */
- private final String columnPrefix;
- private final byte[] columnPrefixBytes;
-
- /**
- * Private constructor, meant to be used by the enum definition.
- *
- * @param columnFamily that this column is stored in.
- * @param columnPrefix for this column.
- */
- private ApplicationColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix) {
- this(columnFamily, columnPrefix, GenericConverter.getInstance());
- }
-
- /**
- * Private constructor, meant to be used by the enum definition.
- *
- * @param columnFamily that this column is stored in.
- * @param columnPrefix for this column.
- * @param converter used to encode/decode values to be stored in HBase for
- * this column prefix.
- */
- private ApplicationColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix, ValueConverter converter) {
- column = new ColumnHelper(columnFamily, converter);
- this.columnFamily = columnFamily;
- this.columnPrefix = columnPrefix;
- if (columnPrefix == null) {
- this.columnPrefixBytes = null;
- } else {
- // Future-proof by ensuring the right column prefix hygiene.
- this.columnPrefixBytes =
- Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
- }
- }
-
- /**
- * @return the column name value
- */
- private String getColumnPrefix() {
- return columnPrefix;
- }
-
- @Override
- public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(
- this.columnPrefixBytes, qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnPrefixBytes(String qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(
- this.columnPrefixBytes, qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnFamilyBytes() {
- return columnFamily.getBytes();
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #store(byte[],
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object)
- */
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, byte[] qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
-
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
-
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- attributes);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #store(byte[],
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object)
- */
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, String qualifier,
- Long timestamp, Object inputValue, Attribute...attributes)
- throws IOException {
-
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
-
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- attributes);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String)
- */
- public Object readResult(Result result, String qualifier) throws IOException {
- byte[] columnQualifier =
- ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
- return column.readResult(result, columnQualifier);
- }
-
- public ValueConverter getValueConverter() {
- return column.getValueConverter();
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResults(org.apache.hadoop.hbase.client.Result,
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
- */
- public Map readResults(Result result,
- KeyConverter keyConverter) throws IOException {
- return column.readResults(result, columnPrefixBytes, keyConverter);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
- */
- public NavigableMap>
- readResultsWithTimestamps(Result result, KeyConverter keyConverter)
- throws IOException {
- return column.readResultsWithTimestamps(result, columnPrefixBytes,
- keyConverter);
- }
-
- /**
- * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there
- * is no match. The following holds true: {@code columnFor(x) == columnFor(y)}
- * if and only if {@code x.equals(y)} or {@code (x == y == null)}
- *
- * @param columnPrefix Name of the column to retrieve
- * @return the corresponding {@link ApplicationColumnPrefix} or null
- */
- public static final ApplicationColumnPrefix columnFor(String columnPrefix) {
-
- // Match column based on value, assume column family matches.
- for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) {
- // Find a match based only on name.
- if (acp.getColumnPrefix().equals(columnPrefix)) {
- return acp;
- }
- }
-
- // Default to null
- return null;
- }
-
- /**
- * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there
- * is no match. The following holds true:
- * {@code columnFor(a,x) == columnFor(b,y)} if and only if
- * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
- *
- * @param columnFamily The columnFamily for which to retrieve the column.
- * @param columnPrefix Name of the column to retrieve
- * @return the corresponding {@link ApplicationColumnPrefix} or null if both
- * arguments don't match.
- */
- public static final ApplicationColumnPrefix columnFor(
- ApplicationColumnFamily columnFamily, String columnPrefix) {
-
- // TODO: needs unit test to confirm and need to update javadoc to explain
- // null prefix case.
-
- for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) {
- // Find a match based column family and on name.
- if (acp.columnFamily.equals(columnFamily)
- && (((columnPrefix == null) && (acp.getColumnPrefix() == null)) ||
- (acp.getColumnPrefix().equals(columnPrefix)))) {
- return acp;
- }
- }
-
- // Default to null
- return null;
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
deleted file mode 100644
index f1e4495..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
+++ /dev/null
@@ -1,206 +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 java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-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.ColumnPrefix;
-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.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * Identifies partially qualified columns for the app-to-flow table.
- */
-public enum AppToFlowColumnPrefix implements ColumnPrefix {
-
- /**
- * The flow name.
- */
- FLOW_NAME(AppToFlowColumnFamily.MAPPING, "flow_name"),
-
- /**
- * The flow run ID.
- */
- FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "flow_run_id"),
-
- /**
- * The user.
- */
- USER_ID(AppToFlowColumnFamily.MAPPING, "user_id");
-
- private final ColumnHelper column;
- private final ColumnFamily columnFamily;
- private final String columnPrefix;
- private final byte[] columnPrefixBytes;
-
- private AppToFlowColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix) {
- this.columnFamily = columnFamily;
- this.columnPrefix = columnPrefix;
- if (columnPrefix == null) {
- this.columnPrefixBytes = null;
- } else {
- // Future-proof by ensuring the right column prefix hygiene.
- this.columnPrefixBytes =
- Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
- }
- this.column = new ColumnHelper(columnFamily);
- }
-
- @Override
- public byte[] getColumnPrefixBytes(String qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(
- columnPrefixBytes, qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(
- columnPrefixBytes, qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnFamilyBytes() {
- return columnFamily.getBytes();
- }
-
- @Override
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, byte[] qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
-
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
-
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- attributes);
- }
-
- @Override
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, String qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
-
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
-
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- attributes);
- }
-
- @Override
- public ValueConverter getValueConverter() {
- return column.getValueConverter();
- }
-
- @Override
- public Object readResult(Result result, String qualifier) throws IOException {
- byte[] columnQualifier =
- ColumnHelper.getColumnQualifier(columnPrefixBytes, qualifier);
- return column.readResult(result, columnQualifier);
- }
-
- @Override
- public Map readResults(Result result,
- KeyConverter keyConverter)
- throws IOException {
- return column.readResults(result, columnPrefixBytes, keyConverter);
- }
-
- @Override
- public NavigableMap>
- readResultsWithTimestamps(Result result,
- KeyConverter keyConverter) throws IOException {
- return column.readResultsWithTimestamps(result, columnPrefixBytes,
- keyConverter);
- }
-
- /**
- * Retrieve an {@link AppToFlowColumnPrefix} given a name, or null if there
- * is no match. The following holds true: {@code columnFor(x) == columnFor(y)}
- * if and only if {@code x.equals(y)} or {@code (x == y == null)}
- *
- * @param columnPrefix Name of the column to retrieve
- * @return the corresponding {@link AppToFlowColumnPrefix} or null
- */
- public static final AppToFlowColumnPrefix columnFor(String columnPrefix) {
-
- // Match column based on value, assume column family matches.
- for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) {
- // Find a match based only on name.
- if (afcp.columnPrefix.equals(columnPrefix)) {
- return afcp;
- }
- }
-
- // Default to null
- return null;
- }
-
- /**
- * Retrieve an {@link AppToFlowColumnPrefix} given a name, or null if there
- * is no match. The following holds true:
- * {@code columnFor(a,x) == columnFor(b,y)} if and only if
- * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
- *
- * @param columnFamily The columnFamily for which to retrieve the column.
- * @param columnPrefix Name of the column to retrieve
- * @return the corresponding {@link AppToFlowColumnPrefix} or null if both
- * arguments don't match.
- */
- public static final AppToFlowColumnPrefix columnFor(
- AppToFlowColumnFamily columnFamily, String columnPrefix) {
-
- // TODO: needs unit test to confirm and need to update javadoc to explain
- // null prefix case.
-
- for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) {
- // Find a match based column family and on name.
- if (afcp.columnFamily.equals(columnFamily)
- && (((columnPrefix == null) && (afcp.columnPrefix == null)) ||
- (afcp.columnPrefix.equals(columnPrefix)))) {
- return afcp;
- }
- }
-
- // Default to null
- return null;
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java
deleted file mode 100644
index 90f2de4..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java
+++ /dev/null
@@ -1,80 +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.common;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * A Column represents the way to store a fully qualified column in a specific
- * table.
- */
-public interface Column {
-
- /**
- * Sends a Mutation to the table. The mutations will be buffered and sent over
- * the wire as part of a batch.
- *
- * @param rowKey identifying the row to write. Nothing gets written when null.
- * @param tableMutator used to modify the underlying HBase table. Caller is
- * responsible to pass a mutator for the table that actually has this
- * column.
- * @param timestamp version timestamp. When null the server timestamp will be
- * used.
- * @param attributes Map of attributes for this mutation. used in the
- * coprocessor to set/read the cell tags. Can be null.
- * @param inputValue the value to write to the rowKey and column qualifier.
- * Nothing gets written when null.
- * @throws IOException if there is any exception encountered during store.
- */
- void store(byte[] rowKey, TypedBufferedMutator tableMutator,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException;
-
- /**
- * Get the latest version of this specified column. Note: this call clones the
- * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
- *
- * @param result Cannot be null
- * @return result object (can be cast to whatever object was written to), or
- * null when result doesn't contain this column.
- * @throws IOException if there is any exception encountered while reading
- * result.
- */
- Object readResult(Result result) throws IOException;
-
- /**
- * Returns column family name(as bytes) associated with this column.
- * @return a byte array encoding column family for this column qualifier.
- */
- byte[] getColumnFamilyBytes();
-
- /**
- * Get byte representation for this column qualifier.
- * @return a byte array representing column qualifier.
- */
- byte[] getColumnQualifierBytes();
-
- /**
- * Returns value converter implementation associated with this column.
- * @return a {@link ValueConverter} implementation.
- */
- ValueConverter getValueConverter();
-}
\ No newline at end of file
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
deleted file mode 100644
index b9815eb..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
+++ /dev/null
@@ -1,389 +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.common;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-/**
- * This class is meant to be used only by explicit Columns, and not directly to
- * write by clients.
- *
- * @param refers to the table.
- */
-public class ColumnHelper {
- private static final Log LOG = LogFactory.getLog(ColumnHelper.class);
-
- private final ColumnFamily columnFamily;
-
- /**
- * Local copy of bytes representation of columnFamily so that we can avoid
- * cloning a new copy over and over.
- */
- private final byte[] columnFamilyBytes;
-
- private final ValueConverter converter;
-
- public ColumnHelper(ColumnFamily columnFamily) {
- this(columnFamily, GenericConverter.getInstance());
- }
-
- public ColumnHelper(ColumnFamily columnFamily, ValueConverter converter) {
- this.columnFamily = columnFamily;
- columnFamilyBytes = columnFamily.getBytes();
- if (converter == null) {
- this.converter = GenericConverter.getInstance();
- } else {
- this.converter = converter;
- }
- }
-
- /**
- * Sends a Mutation to the table. The mutations will be buffered and sent over
- * the wire as part of a batch.
- *
- * @param rowKey
- * identifying the row to write. Nothing gets written when null.
- * @param tableMutator
- * used to modify the underlying HBase table
- * @param columnQualifier
- * column qualifier. Nothing gets written when null.
- * @param timestamp
- * version timestamp. When null the current timestamp multiplied with
- * TimestampGenerator.TS_MULTIPLIER and added with last 3 digits of
- * app id will be used
- * @param inputValue
- * the value to write to the rowKey and column qualifier. Nothing
- * gets written when null.
- * @param attributes Attributes to be set for HBase Put.
- * @throws IOException if any problem occurs during store operation(sending
- * mutation to table).
- */
- public void store(byte[] rowKey, TypedBufferedMutator> tableMutator,
- byte[] columnQualifier, Long timestamp, Object inputValue,
- Attribute... attributes) throws IOException {
- if ((rowKey == null) || (columnQualifier == null) || (inputValue == null)) {
- return;
- }
- Put p = new Put(rowKey);
- timestamp = getPutTimestamp(timestamp, attributes);
- p.addColumn(columnFamilyBytes, columnQualifier, timestamp,
- converter.encodeValue(inputValue));
- if ((attributes != null) && (attributes.length > 0)) {
- for (Attribute attribute : attributes) {
- p.setAttribute(attribute.getName(), attribute.getValue());
- }
- }
- tableMutator.mutate(p);
- }
-
- /*
- * Figures out the cell timestamp used in the Put For storing into flow run
- * table. We would like to left shift the timestamp and supplement it with the
- * AppId id so that there are no collisions in the flow run table's cells
- */
- private long getPutTimestamp(Long timestamp, Attribute[] attributes) {
- if (timestamp == null) {
- timestamp = System.currentTimeMillis();
- }
- String appId = getAppIdFromAttributes(attributes);
- long supplementedTS = TimestampGenerator.getSupplementedTimestamp(
- timestamp, appId);
- return supplementedTS;
- }
-
- private String getAppIdFromAttributes(Attribute[] attributes) {
- if (attributes == null) {
- return null;
- }
- String appId = null;
- for (Attribute attribute : attributes) {
- if (AggregationCompactionDimension.APPLICATION_ID.toString().equals(
- attribute.getName())) {
- appId = Bytes.toString(attribute.getValue());
- }
- }
- return appId;
- }
-
- /**
- * @return the column family for this column implementation.
- */
- public ColumnFamily getColumnFamily() {
- return columnFamily;
- }
-
- /**
- * Get the latest version of this specified column. Note: this call clones the
- * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
- *
- * @param result from which to read the value. Cannot be null
- * @param columnQualifierBytes referring to the column to be read.
- * @return latest version of the specified column of whichever object was
- * written.
- * @throws IOException if any problem occurs while reading result.
- */
- public Object readResult(Result result, byte[] columnQualifierBytes)
- throws IOException {
- if (result == null || columnQualifierBytes == null) {
- return null;
- }
-
- // Would have preferred to be able to use getValueAsByteBuffer and get a
- // ByteBuffer to avoid copy, but GenericObjectMapper doesn't seem to like
- // that.
- byte[] value = result.getValue(columnFamilyBytes, columnQualifierBytes);
- return converter.decodeValue(value);
- }
-
- /**
- * @param result from which to reads data with timestamps
- * @param columnPrefixBytes optional prefix to limit columns. If null all
- * columns are returned.
- * @param identifies the type of column name(indicated by type of key
- * converter).
- * @param the type of the values. The values will be cast into that type.
- * @param keyConverter used to convert column bytes to the appropriate key
- * type.
- * @return the cell values at each respective time in for form
- * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2},
- * idB={timestamp3->value3}, idC={timestamp1->value4}}}
- * @throws IOException if any problem occurs while reading results.
- */
- @SuppressWarnings("unchecked")
- public NavigableMap>
- readResultsWithTimestamps(Result result, byte[] columnPrefixBytes,
- KeyConverter keyConverter) throws IOException {
-
- NavigableMap> results = new TreeMap<>();
-
- if (result != null) {
- NavigableMap<
- byte[], NavigableMap>> resultMap =
- result.getMap();
-
- NavigableMap> columnCellMap =
- resultMap.get(columnFamilyBytes);
-
- // could be that there is no such column family.
- if (columnCellMap != null) {
- for (Entry> entry : columnCellMap
- .entrySet()) {
- K converterColumnKey = null;
- if (columnPrefixBytes == null) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("null prefix was specified; returning all columns");
- }
- try {
- converterColumnKey = keyConverter.decode(entry.getKey());
- } catch (IllegalArgumentException iae) {
- LOG.error("Illegal column found, skipping this column.", iae);
- continue;
- }
- } else {
- // A non-null prefix means columns are actually of the form
- // prefix!columnNameRemainder
- byte[][] columnNameParts =
- Separator.QUALIFIERS.split(entry.getKey(), 2);
- byte[] actualColumnPrefixBytes = columnNameParts[0];
- if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes)
- && columnNameParts.length == 2) {
- try {
- // This is the prefix that we want
- converterColumnKey = keyConverter.decode(columnNameParts[1]);
- } catch (IllegalArgumentException iae) {
- LOG.error("Illegal column found, skipping this column.", iae);
- continue;
- }
- }
- }
-
- // If this column has the prefix we want
- if (converterColumnKey != null) {
- NavigableMap cellResults =
- new TreeMap();
- NavigableMap cells = entry.getValue();
- if (cells != null) {
- for (Entry cell : cells.entrySet()) {
- V value =
- (V) converter.decodeValue(cell.getValue());
- cellResults.put(
- TimestampGenerator.getTruncatedTimestamp(cell.getKey()),
- value);
- }
- }
- results.put(converterColumnKey, cellResults);
- }
- } // for entry : columnCellMap
- } // if columnCellMap != null
- } // if result != null
- return results;
- }
-
- /**
- * @param identifies the type of column name(indicated by type of key
- * converter).
- * @param result from which to read columns
- * @param columnPrefixBytes optional prefix to limit columns. If null all
- * columns are returned.
- * @param keyConverter used to convert column bytes to the appropriate key
- * type.
- * @return the latest values of columns in the column family. If the column
- * prefix is null, the column qualifier is returned as Strings. For a
- * non-null column prefix bytes, the column qualifier is returned as
- * a list of parts, each part a byte[]. This is to facilitate
- * returning byte arrays of values that were not Strings.
- * @throws IOException if any problem occurs while reading results.
- */
- public Map readResults(Result result,
- byte[] columnPrefixBytes, KeyConverter keyConverter)
- throws IOException {
- Map results = new HashMap();
-
- if (result != null) {
- Map columns = result.getFamilyMap(columnFamilyBytes);
- for (Entry entry : columns.entrySet()) {
- byte[] columnKey = entry.getKey();
- if (columnKey != null && columnKey.length > 0) {
-
- K converterColumnKey = null;
- if (columnPrefixBytes == null) {
- try {
- converterColumnKey = keyConverter.decode(columnKey);
- } catch (IllegalArgumentException iae) {
- LOG.error("Illegal column found, skipping this column.", iae);
- continue;
- }
- } else {
- // A non-null prefix means columns are actually of the form
- // prefix!columnNameRemainder
- byte[][] columnNameParts = Separator.QUALIFIERS.split(columnKey, 2);
- if (columnNameParts.length > 0) {
- byte[] actualColumnPrefixBytes = columnNameParts[0];
- // If this is the prefix that we want
- if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes)
- && columnNameParts.length == 2) {
- try {
- converterColumnKey = keyConverter.decode(columnNameParts[1]);
- } catch (IllegalArgumentException iae) {
- LOG.error("Illegal column found, skipping this column.", iae);
- continue;
- }
- }
- }
- } // if-else
-
- // If the columnPrefix is null (we want all columns), or the actual
- // prefix matches the given prefix we want this column
- if (converterColumnKey != null) {
- Object value = converter.decodeValue(entry.getValue());
- // we return the columnQualifier in parts since we don't know
- // which part is of which data type.
- results.put(converterColumnKey, value);
- }
- }
- } // for entry
- }
- return results;
- }
-
- /**
- * @param columnPrefixBytes The byte representation for the column prefix.
- * Should not contain {@link Separator#QUALIFIERS}.
- * @param qualifier for the remainder of the column.
- * {@link Separator#QUALIFIERS} is permissible in the qualifier
- * as it is joined only with the column prefix bytes.
- * @return fully sanitized column qualifier that is a combination of prefix
- * and qualifier. If prefix is null, the result is simply the encoded
- * qualifier without any separator.
- */
- public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
- String qualifier) {
-
- // We don't want column names to have spaces / tabs.
- byte[] encodedQualifier =
- Separator.encode(qualifier, Separator.SPACE, Separator.TAB);
- if (columnPrefixBytes == null) {
- return encodedQualifier;
- }
-
- // Convert qualifier to lower case, strip of separators and tag on column
- // prefix.
- byte[] columnQualifier =
- Separator.QUALIFIERS.join(columnPrefixBytes, encodedQualifier);
- return columnQualifier;
- }
-
- /**
- * @param columnPrefixBytes The byte representation for the column prefix.
- * Should not contain {@link Separator#QUALIFIERS}.
- * @param qualifier for the remainder of the column.
- * @return fully sanitized column qualifier that is a combination of prefix
- * and qualifier. If prefix is null, the result is simply the encoded
- * qualifier without any separator.
- */
- public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
- long qualifier) {
-
- if (columnPrefixBytes == null) {
- return Bytes.toBytes(qualifier);
- }
-
- // Convert qualifier to lower case, strip of separators and tag on column
- // prefix.
- byte[] columnQualifier =
- Separator.QUALIFIERS.join(columnPrefixBytes, Bytes.toBytes(qualifier));
- return columnQualifier;
- }
-
- public ValueConverter getValueConverter() {
- return converter;
- }
-
- /**
- * @param columnPrefixBytes The byte representation for the column prefix.
- * Should not contain {@link Separator#QUALIFIERS}.
- * @param qualifier the byte representation for the remainder of the column.
- * @return fully sanitized column qualifier that is a combination of prefix
- * and qualifier. If prefix is null, the result is simply the encoded
- * qualifier without any separator.
- */
- public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
- byte[] qualifier) {
-
- if (columnPrefixBytes == null) {
- return qualifier;
- }
-
- byte[] columnQualifier =
- Separator.QUALIFIERS.join(columnPrefixBytes, qualifier);
- return columnQualifier;
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java
deleted file mode 100644
index 89aa013..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java
+++ /dev/null
@@ -1,145 +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.common;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * Used to represent a partially qualified column, where the actual column name
- * will be composed of a prefix and the remainder of the column qualifier. The
- * prefix can be null, in which case the column qualifier will be completely
- * determined when the values are stored.
- */
-public interface ColumnPrefix {
-
- /**
- * Sends a Mutation to the table. The mutations will be buffered and sent over
- * the wire as part of a batch.
- *
- * @param rowKey identifying the row to write. Nothing gets written when null.
- * @param tableMutator used to modify the underlying HBase table. Caller is
- * responsible to pass a mutator for the table that actually has this
- * column.
- * @param qualifier column qualifier. Nothing gets written when null.
- * @param timestamp version timestamp. When null the server timestamp will be
- * used.
- * @param attributes attributes for the mutation that are used by the
- * coprocessor to set/read the cell tags.
- * @param inputValue the value to write to the rowKey and column qualifier.
- * Nothing gets written when null.
- * @throws IOException if there is any exception encountered while doing
- * store operation(sending mutation to the table).
- */
- void store(byte[] rowKey, TypedBufferedMutator tableMutator,
- byte[] qualifier, Long timestamp, Object inputValue,
- Attribute... attributes) throws IOException;
-
- /**
- * Sends a Mutation to the table. The mutations will be buffered and sent over
- * the wire as part of a batch.
- *
- * @param rowKey identifying the row to write. Nothing gets written when null.
- * @param tableMutator used to modify the underlying HBase table. Caller is
- * responsible to pass a mutator for the table that actually has this
- * column.
- * @param qualifier column qualifier. Nothing gets written when null.
- * @param timestamp version timestamp. When null the server timestamp will be
- * used.
- * @param attributes attributes for the mutation that are used by the
- * coprocessor to set/read the cell tags.
- * @param inputValue the value to write to the rowKey and column qualifier.
- * Nothing gets written when null.
- * @throws IOException if there is any exception encountered while doing
- * store operation(sending mutation to the table).
- */
- void store(byte[] rowKey, TypedBufferedMutator tableMutator,
- String qualifier, Long timestamp, Object inputValue,
- Attribute... attributes) throws IOException;
-
- /**
- * Get the latest version of this specified column. Note: this call clones the
- * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
- *
- * @param result Cannot be null
- * @param qualifier column qualifier. Nothing gets read when null.
- * @return result object (can be cast to whatever object was written to) or
- * null when specified column qualifier for this prefix doesn't exist
- * in the result.
- * @throws IOException if there is any exception encountered while reading
- * result.
- */
- Object readResult(Result result, String qualifier) throws IOException;
-
- /**
- *
- * @param identifies the type of key converter.
- * @param result from which to read columns.
- * @param keyConverter used to convert column bytes to the appropriate key
- * type
- * @return the latest values of columns in the column family with this prefix
- * (or all of them if the prefix value is null).
- * @throws IOException if there is any exception encountered while reading
- * results.
- */
- Map readResults(Result result, KeyConverter keyConverter)
- throws IOException;
-
- /**
- * @param result from which to reads data with timestamps.
- * @param identifies the type of key converter.
- * @param the type of the values. The values will be cast into that type.
- * @param keyConverter used to convert column bytes to the appropriate key
- * type.
- * @return the cell values at each respective time in for form
- * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2},
- * idB={timestamp3->value3}, idC={timestamp1->value4}}}
- * @throws IOException if there is any exception encountered while reading
- * result.
- */
- NavigableMap> readResultsWithTimestamps(
- Result result, KeyConverter keyConverter) throws IOException;
-
- /**
- * @param qualifierPrefix Column qualifier or prefix of qualifier.
- * @return a byte array encoding column prefix and qualifier/prefix passed.
- */
- byte[] getColumnPrefixBytes(String qualifierPrefix);
-
- /**
- * @param qualifierPrefix Column qualifier or prefix of qualifier.
- * @return a byte array encoding column prefix and qualifier/prefix passed.
- */
- byte[] getColumnPrefixBytes(byte[] qualifierPrefix);
-
- /**
- * Returns column family name(as bytes) associated with this column prefix.
- * @return a byte array encoding column family for this prefix.
- */
- byte[] getColumnFamilyBytes();
-
- /**
- * Returns value converter implementation associated with this column prefix.
- * @return a {@link ValueConverter} implementation.
- */
- ValueConverter getValueConverter();
-}
\ No newline at end of file
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
deleted file mode 100644
index afe4d6a..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
+++ /dev/null
@@ -1,306 +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.common;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-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;
-
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A bunch of utility functions used in HBase TimelineService backend.
- */
-public final class HBaseTimelineStorageUtils {
-
- /** milliseconds in one day. */
- public static final long MILLIS_ONE_DAY = 86400000L;
-
- private HBaseTimelineStorageUtils() {
- }
-
- /**
- * Combines the input array of attributes and the input aggregation operation
- * into a new array of attributes.
- *
- * @param attributes Attributes to be combined.
- * @param aggOp Aggregation operation.
- * @return array of combined attributes.
- */
- public static Attribute[] combineAttributes(Attribute[] attributes,
- AggregationOperation aggOp) {
- int newLength = getNewLengthCombinedAttributes(attributes, aggOp);
- Attribute[] combinedAttributes = new Attribute[newLength];
-
- if (attributes != null) {
- System.arraycopy(attributes, 0, combinedAttributes, 0, attributes.length);
- }
-
- if (aggOp != null) {
- Attribute a2 = aggOp.getAttribute();
- combinedAttributes[newLength - 1] = a2;
- }
- return combinedAttributes;
- }
-
- /**
- * Returns a number for the new array size. The new array is the combination
- * of input array of attributes and the input aggregation operation.
- *
- * @param attributes Attributes.
- * @param aggOp Aggregation operation.
- * @return the size for the new array
- */
- private static int getNewLengthCombinedAttributes(Attribute[] attributes,
- AggregationOperation aggOp) {
- int oldLength = getAttributesLength(attributes);
- int aggLength = getAppOpLength(aggOp);
- return oldLength + aggLength;
- }
-
- private static int getAppOpLength(AggregationOperation aggOp) {
- if (aggOp != null) {
- return 1;
- }
- return 0;
- }
-
- private static int getAttributesLength(Attribute[] attributes) {
- if (attributes != null) {
- return attributes.length;
- }
- return 0;
- }
-
- /**
- * Returns the first seen aggregation operation as seen in the list of input
- * tags or null otherwise.
- *
- * @param tags list of HBase tags.
- * @return AggregationOperation
- */
- public static AggregationOperation getAggregationOperationFromTagsList(
- List tags) {
- for (AggregationOperation aggOp : AggregationOperation.values()) {
- for (Tag tag : tags) {
- if (tag.getType() == aggOp.getTagType()) {
- return aggOp;
- }
- }
- }
- return null;
- }
-
- /**
- * Creates a {@link Tag} from the input attribute.
- *
- * @param attribute Attribute from which tag has to be fetched.
- * @return a HBase Tag.
- */
- public static Tag getTagFromAttribute(Map.Entry attribute) {
- // attribute could be either an Aggregation Operation or
- // an Aggregation Dimension
- // Get the Tag type from either
- AggregationOperation aggOp = AggregationOperation
- .getAggregationOperation(attribute.getKey());
- if (aggOp != null) {
- Tag t = new Tag(aggOp.getTagType(), attribute.getValue());
- return t;
- }
-
- AggregationCompactionDimension aggCompactDim =
- AggregationCompactionDimension.getAggregationCompactionDimension(
- attribute.getKey());
- if (aggCompactDim != null) {
- Tag t = new Tag(aggCompactDim.getTagType(), attribute.getValue());
- return t;
- }
- return null;
- }
-
- /**
- * creates a new cell based on the input cell but with the new value.
- *
- * @param origCell Original cell
- * @param newValue new cell value
- * @return cell
- * @throws IOException while creating new cell.
- */
- public static Cell createNewCell(Cell origCell, byte[] newValue)
- throws IOException {
- return CellUtil.createCell(CellUtil.cloneRow(origCell),
- CellUtil.cloneFamily(origCell), CellUtil.cloneQualifier(origCell),
- origCell.getTimestamp(), KeyValue.Type.Put.getCode(), newValue);
- }
-
- /**
- * creates a cell with the given inputs.
- *
- * @param row row of the cell to be created
- * @param family column family name of the new cell
- * @param qualifier qualifier for the new cell
- * @param ts timestamp of the new cell
- * @param newValue value of the new cell
- * @param tags tags in the new cell
- * @return cell
- * @throws IOException while creating the cell.
- */
- public static Cell createNewCell(byte[] row, byte[] family, byte[] qualifier,
- long ts, byte[] newValue, byte[] tags) throws IOException {
- return CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put,
- newValue, tags);
- }
-
- /**
- * returns app id from the list of tags.
- *
- * @param tags cell tags to be looked into
- * @return App Id as the AggregationCompactionDimension
- */
- public static String getAggregationCompactionDimension(List tags) {
- String appId = null;
- for (Tag t : tags) {
- if (AggregationCompactionDimension.APPLICATION_ID.getTagType() == t
- .getType()) {
- appId = Bytes.toString(t.getValue());
- return appId;
- }
- }
- return appId;
- }
-
- /**
- * 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
- * (scans start at the largest int first).
- *
- * @param key value to be inverted so that the latest version will be first in
- * a scan.
- * @return inverted int
- */
- public static int invertInt(int key) {
- return Integer.MAX_VALUE - key;
- }
-
- /**
- * returns the timestamp of that day's start (which is midnight 00:00:00 AM)
- * for a given input timestamp.
- *
- * @param ts Timestamp.
- * @return timestamp of that day's beginning (midnight)
- */
- public static long getTopOfTheDayTimestamp(long ts) {
- long dayTimestamp = ts - (ts % MILLIS_ONE_DAY);
- return dayTimestamp;
- }
-
- /**
- * @param conf Yarn configuration. Used to see if there is an explicit config
- * pointing to the HBase config file to read. If null then a new
- * HBase configuration will be returned.
- * @return a configuration with the HBase configuration from the classpath,
- * optionally overwritten by the timeline service configuration URL if
- * specified.
- * @throws MalformedURLException if a timeline service HBase configuration URL
- * is specified but is a malformed URL.
- */
- public static Configuration getTimelineServiceHBaseConf(Configuration conf)
- throws MalformedURLException {
- Configuration hbaseConf;
-
- if (conf == null) {
- return HBaseConfiguration.create();
- }
-
- String timelineServiceHBaseConfFileURL =
- conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE);
- if (timelineServiceHBaseConfFileURL != null
- && timelineServiceHBaseConfFileURL.length() > 0) {
- // create a clone so that we don't mess with out input one
- hbaseConf = new Configuration(conf);
- Configuration plainHBaseConf = new Configuration(false);
- URL hbaseSiteXML = new URL(timelineServiceHBaseConfFileURL);
- plainHBaseConf.addResource(hbaseSiteXML);
- HBaseConfiguration.merge(hbaseConf, plainHBaseConf);
- } else {
- // default to what is on the classpath
- hbaseConf = HBaseConfiguration.create(conf);
- }
- return hbaseConf;
- }
-
- /**
- * Given a row key prefix stored in a byte array, return a byte array for its
- * immediate next row key.
- *
- * @param rowKeyPrefix The provided row key prefix, represented in an array.
- * @return the closest next row key of the provided row key.
- */
- public static byte[] calculateTheClosestNextRowKeyForPrefix(
- byte[] rowKeyPrefix) {
- // Essentially we are treating it like an 'unsigned very very long' and
- // doing +1 manually.
- // Search for the place where the trailing 0xFFs start
- int offset = rowKeyPrefix.length;
- while (offset > 0) {
- if (rowKeyPrefix[offset - 1] != (byte) 0xFF) {
- break;
- }
- offset--;
- }
-
- if (offset == 0) {
- // We got an 0xFFFF... (only FFs) stopRow value which is
- // the last possible prefix before the end of the table.
- // So set it to stop at the 'end of the table'
- return HConstants.EMPTY_END_ROW;
- }
-
- // Copy the right length of the original
- byte[] newStopRow = Arrays.copyOfRange(rowKeyPrefix, 0, offset);
- // And increment the last one
- newStopRow[newStopRow.length - 1]++;
- return newStopRow;
- }
-
- /**
- * Checks if passed object is of integral type(Short/Integer/Long).
- *
- * @param obj Object to be checked.
- * @return true if object passed is of type Short or Integer or Long, false
- * otherwise.
- */
- public static boolean isIntegralValue(Object obj) {
- return (obj instanceof Short) || (obj instanceof Integer) ||
- (obj instanceof Long);
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
deleted file mode 100644
index 93b4b36..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
+++ /dev/null
@@ -1,160 +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 java.io.IOException;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-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.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * Identifies fully qualified columns for the {@link EntityTable}.
- */
-public enum EntityColumn implements Column {
-
- /**
- * Identifier for the entity.
- */
- ID(EntityColumnFamily.INFO, "id"),
-
- /**
- * The type of entity.
- */
- TYPE(EntityColumnFamily.INFO, "type"),
-
- /**
- * When the entity was created.
- */
- CREATED_TIME(EntityColumnFamily.INFO, "created_time", new LongConverter()),
-
- /**
- * The version of the flow that this entity belongs to.
- */
- FLOW_VERSION(EntityColumnFamily.INFO, "flow_version");
-
- private final ColumnHelper column;
- private final ColumnFamily columnFamily;
- private final String columnQualifier;
- private final byte[] columnQualifierBytes;
-
- EntityColumn(ColumnFamily columnFamily,
- String columnQualifier) {
- this(columnFamily, columnQualifier, GenericConverter.getInstance());
- }
-
- EntityColumn(ColumnFamily columnFamily,
- String columnQualifier, ValueConverter converter) {
- this.columnFamily = columnFamily;
- this.columnQualifier = columnQualifier;
- // Future-proof by ensuring the right column prefix hygiene.
- this.columnQualifierBytes =
- Bytes.toBytes(Separator.SPACE.encode(columnQualifier));
- this.column = new ColumnHelper(columnFamily, converter);
- }
-
- /**
- * @return the column name value
- */
- private String getColumnQualifier() {
- return columnQualifier;
- }
-
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, Long timestamp,
- Object inputValue, Attribute... attributes) throws IOException {
- column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
- inputValue, attributes);
- }
-
- public Object readResult(Result result) throws IOException {
- return column.readResult(result, columnQualifierBytes);
- }
-
- /**
- * Retrieve an {@link EntityColumn} given a name, or null if there is no
- * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
- * and only if {@code x.equals(y)} or {@code (x == y == null)}
- *
- * @param columnQualifier Name of the column to retrieve
- * @return the corresponding {@link EntityColumn} or null
- */
- public static final EntityColumn columnFor(String columnQualifier) {
-
- // Match column based on value, assume column family matches.
- for (EntityColumn ec : EntityColumn.values()) {
- // Find a match based only on name.
- if (ec.getColumnQualifier().equals(columnQualifier)) {
- return ec;
- }
- }
-
- // Default to null
- return null;
- }
-
- @Override
- public byte[] getColumnQualifierBytes() {
- return columnQualifierBytes.clone();
- }
-
- @Override
- public byte[] getColumnFamilyBytes() {
- return columnFamily.getBytes();
- }
-
- @Override
- public ValueConverter getValueConverter() {
- return column.getValueConverter();
- }
-
- /**
- * Retrieve an {@link EntityColumn} given a name, or null if there is no
- * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
- * if and only if {@code a.equals(b) & x.equals(y)} or
- * {@code (x == y == null)}
- *
- * @param columnFamily The columnFamily for which to retrieve the column.
- * @param name Name of the column to retrieve
- * @return the corresponding {@link EntityColumn} or null if both arguments
- * don't match.
- */
- public static final EntityColumn columnFor(EntityColumnFamily columnFamily,
- String name) {
-
- for (EntityColumn ec : EntityColumn.values()) {
- // Find a match based column family and on name.
- if (ec.columnFamily.equals(columnFamily)
- && ec.getColumnQualifier().equals(name)) {
- return ec;
- }
- }
-
- // Default to null
- return null;
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
deleted file mode 100644
index e410549..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
+++ /dev/null
@@ -1,300 +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 java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-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.ColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
-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.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * Identifies partially qualified columns for the entity table.
- */
-public enum EntityColumnPrefix implements ColumnPrefix {
-
- /**
- * To store TimelineEntity getIsRelatedToEntities values.
- */
- IS_RELATED_TO(EntityColumnFamily.INFO, "s"),
-
- /**
- * To store TimelineEntity getRelatesToEntities values.
- */
- RELATES_TO(EntityColumnFamily.INFO, "r"),
-
- /**
- * To store TimelineEntity info values.
- */
- INFO(EntityColumnFamily.INFO, "i"),
-
- /**
- * Lifecycle events for an entity.
- */
- EVENT(EntityColumnFamily.INFO, "e", true),
-
- /**
- * Config column stores configuration with config key as the column name.
- */
- CONFIG(EntityColumnFamily.CONFIGS, null),
-
- /**
- * Metrics are stored with the metric name as the column name.
- */
- METRIC(EntityColumnFamily.METRICS, null, new LongConverter());
-
- private final ColumnHelper column;
- private final ColumnFamily columnFamily;
-
- /**
- * Can be null for those cases where the provided column qualifier is the
- * entire column name.
- */
- private final String columnPrefix;
- private final byte[] columnPrefixBytes;
-
- /**
- * Private constructor, meant to be used by the enum definition.
- *
- * @param columnFamily that this column is stored in.
- * @param columnPrefix for this column.
- */
- EntityColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix) {
- this(columnFamily, columnPrefix, false, GenericConverter.getInstance());
- }
-
- EntityColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix, boolean compondColQual) {
- this(columnFamily, columnPrefix, compondColQual,
- GenericConverter.getInstance());
- }
-
- EntityColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix, ValueConverter converter) {
- this(columnFamily, columnPrefix, false, converter);
- }
-
- /**
- * Private constructor, meant to be used by the enum definition.
- *
- * @param columnFamily that this column is stored in.
- * @param columnPrefix for this column.
- * @param converter used to encode/decode values to be stored in HBase for
- * this column prefix.
- */
- EntityColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix, boolean compondColQual, ValueConverter converter) {
- column = new ColumnHelper(columnFamily, converter);
- this.columnFamily = columnFamily;
- this.columnPrefix = columnPrefix;
- if (columnPrefix == null) {
- this.columnPrefixBytes = null;
- } else {
- // Future-proof by ensuring the right column prefix hygiene.
- this.columnPrefixBytes =
- Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
- }
- }
-
- /**
- * @return the column name value
- */
- public String getColumnPrefix() {
- return columnPrefix;
- }
-
- @Override
- public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(
- this.columnPrefixBytes, qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnPrefixBytes(String qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(
- this.columnPrefixBytes, qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnFamilyBytes() {
- return columnFamily.getBytes();
- }
-
- @Override
- public ValueConverter getValueConverter() {
- return column.getValueConverter();
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #store(byte[],
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object,
- * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[])
- */
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, String qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
-
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
-
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- attributes);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #store(byte[],
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object)
- */
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, byte[] qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
-
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
-
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- attributes);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String)
- */
- public Object readResult(Result result, String qualifier) throws IOException {
- byte[] columnQualifier =
- ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
- return column.readResult(result, columnQualifier);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResults(org.apache.hadoop.hbase.client.Result,
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
- */
- public Map readResults(Result result,
- KeyConverter keyConverter) throws IOException {
- return column.readResults(result, columnPrefixBytes, keyConverter);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
- */
- public NavigableMap>
- readResultsWithTimestamps(Result result, KeyConverter keyConverter)
- throws IOException {
- return column.readResultsWithTimestamps(result, columnPrefixBytes,
- keyConverter);
- }
-
- /**
- * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no
- * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
- * and only if {@code x.equals(y)} or {@code (x == y == null)}
- *
- * @param columnPrefix Name of the column to retrieve
- * @return the corresponding {@link EntityColumnPrefix} or null
- */
- public static final EntityColumnPrefix columnFor(String columnPrefix) {
-
- // Match column based on value, assume column family matches.
- for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) {
- // Find a match based only on name.
- if (ecp.getColumnPrefix().equals(columnPrefix)) {
- return ecp;
- }
- }
-
- // Default to null
- return null;
- }
-
- /**
- * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no
- * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
- * if and only if {@code (x == y == null)} or
- * {@code a.equals(b) & x.equals(y)}
- *
- * @param columnFamily The columnFamily for which to retrieve the column.
- * @param columnPrefix Name of the column to retrieve
- * @return the corresponding {@link EntityColumnPrefix} or null if both
- * arguments don't match.
- */
- public static final EntityColumnPrefix columnFor(
- EntityColumnFamily columnFamily, String columnPrefix) {
-
- // TODO: needs unit test to confirm and need to update javadoc to explain
- // null prefix case.
-
- for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) {
- // Find a match based column family and on name.
- if (ecp.columnFamily.equals(columnFamily)
- && (((columnPrefix == null) && (ecp.getColumnPrefix() == null)) ||
- (ecp.getColumnPrefix().equals(columnPrefix)))) {
- return ecp;
- }
- }
-
- // Default to null
- return null;
- }
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
deleted file mode 100644
index 439e0c8..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
+++ /dev/null
@@ -1,277 +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 java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-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.ColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
-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.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-
-/**
- * Identifies partially qualified columns for the {@link FlowActivityTable}.
- */
-public enum FlowActivityColumnPrefix
- implements ColumnPrefix {
-
- /**
- * To store run ids of the flows.
- */
- RUN_ID(FlowActivityColumnFamily.INFO, "r", null);
-
- private final ColumnHelper column;
- private final ColumnFamily columnFamily;
-
- /**
- * Can be null for those cases where the provided column qualifier is the
- * entire column name.
- */
- private final String columnPrefix;
- private final byte[] columnPrefixBytes;
-
- private final AggregationOperation aggOp;
-
- /**
- * Private constructor, meant to be used by the enum definition.
- *
- * @param columnFamily
- * that this column is stored in.
- * @param columnPrefix
- * for this column.
- */
- private FlowActivityColumnPrefix(
- ColumnFamily columnFamily, String columnPrefix,
- AggregationOperation aggOp) {
- this(columnFamily, columnPrefix, aggOp, false);
- }
-
- private FlowActivityColumnPrefix(
- ColumnFamily columnFamily, String columnPrefix,
- AggregationOperation aggOp, boolean compoundColQual) {
- column = new ColumnHelper(columnFamily);
- this.columnFamily = columnFamily;
- this.columnPrefix = columnPrefix;
- if (columnPrefix == null) {
- this.columnPrefixBytes = null;
- } else {
- // Future-proof by ensuring the right column prefix hygiene.
- this.columnPrefixBytes = Bytes.toBytes(Separator.SPACE
- .encode(columnPrefix));
- }
- this.aggOp = aggOp;
- }
-
- /**
- * @return the column name value
- */
- public String getColumnPrefix() {
- return columnPrefix;
- }
-
- @Override
- public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(
- this.columnPrefixBytes, qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnPrefixBytes(String qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(
- this.columnPrefixBytes, qualifierPrefix);
- }
-
- public byte[] getColumnPrefixBytes() {
- return columnPrefixBytes.clone();
- }
-
- @Override
- public byte[] getColumnFamilyBytes() {
- return columnFamily.getBytes();
- }
-
- @Override
- public ValueConverter getValueConverter() {
- return column.getValueConverter();
- }
-
- public AggregationOperation getAttribute() {
- return aggOp;
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #store(byte[],
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, byte[], java.lang.Long, java.lang.Object,
- * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[])
- */
- @Override
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, byte[] qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
- Attribute[] combinedAttributes =
- HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp);
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- combinedAttributes);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String)
- */
- public Object readResult(Result result, String qualifier) throws IOException {
- byte[] columnQualifier = ColumnHelper.getColumnQualifier(
- this.columnPrefixBytes, qualifier);
- return column.readResult(result, columnQualifier);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResults(org.apache.hadoop.hbase.client.Result,
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
- */
- public Map readResults(Result result,
- KeyConverter keyConverter) throws IOException {
- return column.readResults(result, columnPrefixBytes, keyConverter);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
- */
- public NavigableMap>
- readResultsWithTimestamps(Result result, KeyConverter keyConverter)
- throws IOException {
- return column.readResultsWithTimestamps(result, columnPrefixBytes,
- keyConverter);
- }
-
- /**
- * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there
- * is no match. The following holds true: {@code columnFor(x) == columnFor(y)}
- * if and only if {@code x.equals(y)} or {@code (x == y == null)}
- *
- * @param columnPrefix
- * Name of the column to retrieve
- * @return the corresponding {@link FlowActivityColumnPrefix} or null
- */
- public static final FlowActivityColumnPrefix columnFor(String columnPrefix) {
-
- // Match column based on value, assume column family matches.
- for (FlowActivityColumnPrefix flowActivityColPrefix :
- FlowActivityColumnPrefix.values()) {
- // Find a match based only on name.
- if (flowActivityColPrefix.getColumnPrefix().equals(columnPrefix)) {
- return flowActivityColPrefix;
- }
- }
- // Default to null
- return null;
- }
-
- /**
- * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there
- * is no match. The following holds true:
- * {@code columnFor(a,x) == columnFor(b,y)} if and only if
- * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
- *
- * @param columnFamily
- * The columnFamily for which to retrieve the column.
- * @param columnPrefix
- * Name of the column to retrieve
- * @return the corresponding {@link FlowActivityColumnPrefix} or null if both
- * arguments don't match.
- */
- public static final FlowActivityColumnPrefix columnFor(
- FlowActivityColumnFamily columnFamily, String columnPrefix) {
-
- // TODO: needs unit test to confirm and need to update javadoc to explain
- // null prefix case.
-
- for (FlowActivityColumnPrefix flowActivityColumnPrefix :
- FlowActivityColumnPrefix.values()) {
- // Find a match based column family and on name.
- if (flowActivityColumnPrefix.columnFamily.equals(columnFamily)
- && (((columnPrefix == null) && (flowActivityColumnPrefix
- .getColumnPrefix() == null)) || (flowActivityColumnPrefix
- .getColumnPrefix().equals(columnPrefix)))) {
- return flowActivityColumnPrefix;
- }
- }
- // Default to null
- return null;
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #store(byte[],
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object,
- * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[])
- */
- @Override
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, String qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
- Attribute[] combinedAttributes =
- HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp);
- column.store(rowKey, tableMutator, columnQualifier, null, inputValue,
- combinedAttributes);
- }
-}
\ No newline at end of file
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
deleted file mode 100644
index 90dd345..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
+++ /dev/null
@@ -1,182 +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 java.io.IOException;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-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.HBaseTimelineStorageUtils;
-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.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-
-/**
- * Identifies fully qualified columns for the {@link FlowRunTable}.
- */
-public enum FlowRunColumn implements Column {
-
- /**
- * When the flow was started. This is the minimum of currently known
- * application start times.
- */
- MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time",
- 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, new LongConverter()),
-
- /**
- * The version of the flow that this flow belongs to.
- */
- FLOW_VERSION(FlowRunColumnFamily.INFO, "flow_version", null);
-
- private final ColumnHelper column;
- private final ColumnFamily columnFamily;
- private final String columnQualifier;
- private final byte[] columnQualifierBytes;
- private final AggregationOperation aggOp;
-
- private FlowRunColumn(ColumnFamily columnFamily,
- String columnQualifier, AggregationOperation aggOp) {
- this(columnFamily, columnQualifier, aggOp,
- GenericConverter.getInstance());
- }
-
- private FlowRunColumn(ColumnFamily columnFamily,
- String columnQualifier, AggregationOperation aggOp,
- ValueConverter converter) {
- this.columnFamily = columnFamily;
- this.columnQualifier = columnQualifier;
- this.aggOp = aggOp;
- // Future-proof by ensuring the right column prefix hygiene.
- this.columnQualifierBytes = Bytes.toBytes(Separator.SPACE
- .encode(columnQualifier));
- this.column = new ColumnHelper(columnFamily, converter);
- }
-
- /**
- * @return the column name value
- */
- private String getColumnQualifier() {
- return columnQualifier;
- }
-
- @Override
- public byte[] getColumnQualifierBytes() {
- return columnQualifierBytes.clone();
- }
-
- @Override
- public byte[] getColumnFamilyBytes() {
- return columnFamily.getBytes();
- }
-
- public AggregationOperation getAggregationOperation() {
- return aggOp;
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.Column#store
- * (byte[], org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, java.lang.Long, java.lang.Object,
- * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[])
- */
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, Long timestamp,
- Object inputValue, Attribute... attributes) throws IOException {
-
- Attribute[] combinedAttributes =
- HBaseTimelineStorageUtils.combineAttributes(attributes, aggOp);
- column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
- inputValue, combinedAttributes);
- }
-
- public Object readResult(Result result) throws IOException {
- return column.readResult(result, columnQualifierBytes);
- }
-
- /**
- * Retrieve an {@link FlowRunColumn} given a name, or null if there is no
- * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
- * and only if {@code x.equals(y)} or {@code (x == y == null)}
- *
- * @param columnQualifier
- * Name of the column to retrieve
- * @return the corresponding {@link FlowRunColumn} or null
- */
- public static final FlowRunColumn columnFor(String columnQualifier) {
-
- // Match column based on value, assume column family matches.
- for (FlowRunColumn ec : FlowRunColumn.values()) {
- // Find a match based only on name.
- if (ec.getColumnQualifier().equals(columnQualifier)) {
- return ec;
- }
- }
-
- // Default to null
- return null;
- }
-
- @Override
- public ValueConverter getValueConverter() {
- return column.getValueConverter();
- }
-
- /**
- * Retrieve an {@link FlowRunColumn} given a name, or null if there is no
- * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
- * if and only if {@code a.equals(b) & x.equals(y)} or
- * {@code (x == y == null)}
- *
- * @param columnFamily
- * The columnFamily for which to retrieve the column.
- * @param name
- * Name of the column to retrieve
- * @return the corresponding {@link FlowRunColumn} or null if both arguments
- * don't match.
- */
- public static final FlowRunColumn columnFor(FlowRunColumnFamily columnFamily,
- String name) {
-
- for (FlowRunColumn ec : FlowRunColumn.values()) {
- // Find a match based column family and on name.
- if (ec.columnFamily.equals(columnFamily)
- && ec.getColumnQualifier().equals(name)) {
- return ec;
- }
- }
-
- // Default to null
- return null;
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
deleted file mode 100644
index 278d18e..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
+++ /dev/null
@@ -1,268 +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 java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-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.ColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
-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.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-
-/**
- * Identifies partially qualified columns for the {@link FlowRunTable}.
- */
-public enum FlowRunColumnPrefix implements ColumnPrefix {
-
- /**
- * To store flow run info values.
- */
- METRIC(FlowRunColumnFamily.INFO, "m", null, new LongConverter());
-
- private final ColumnHelper column;
- private final ColumnFamily columnFamily;
-
- /**
- * Can be null for those cases where the provided column qualifier is the
- * entire column name.
- */
- private final String columnPrefix;
- private final byte[] columnPrefixBytes;
-
- private final AggregationOperation aggOp;
-
- /**
- * Private constructor, meant to be used by the enum definition.
- *
- * @param columnFamily that this column is stored in.
- * @param columnPrefix for this column.
- */
- private FlowRunColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix, AggregationOperation fra, ValueConverter converter) {
- this(columnFamily, columnPrefix, fra, converter, false);
- }
-
- private FlowRunColumnPrefix(ColumnFamily columnFamily,
- String columnPrefix, AggregationOperation fra, ValueConverter converter,
- boolean compoundColQual) {
- column = new ColumnHelper(columnFamily, converter);
- this.columnFamily = columnFamily;
- this.columnPrefix = columnPrefix;
- if (columnPrefix == null) {
- this.columnPrefixBytes = null;
- } else {
- // Future-proof by ensuring the right column prefix hygiene.
- this.columnPrefixBytes =
- Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
- }
- this.aggOp = fra;
- }
-
- /**
- * @return the column name value
- */
- public String getColumnPrefix() {
- return columnPrefix;
- }
-
- public byte[] getColumnPrefixBytes() {
- return columnPrefixBytes.clone();
- }
-
- @Override
- public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(this.columnPrefixBytes,
- qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnPrefixBytes(String qualifierPrefix) {
- return ColumnHelper.getColumnQualifier(this.columnPrefixBytes,
- qualifierPrefix);
- }
-
- @Override
- public byte[] getColumnFamilyBytes() {
- return columnFamily.getBytes();
- }
-
- public AggregationOperation getAttribute() {
- return aggOp;
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #store(byte[],
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object)
- */
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, String qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
-
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
- Attribute[] combinedAttributes =
- HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp);
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- combinedAttributes);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #store(byte[],
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.
- * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object)
- */
- public void store(byte[] rowKey,
- TypedBufferedMutator tableMutator, byte[] qualifier,
- Long timestamp, Object inputValue, Attribute... attributes)
- throws IOException {
-
- // Null check
- if (qualifier == null) {
- throw new IOException("Cannot store column with null qualifier in "
- + tableMutator.getName().getNameAsString());
- }
-
- byte[] columnQualifier = getColumnPrefixBytes(qualifier);
- Attribute[] combinedAttributes =
- HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp);
- column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
- combinedAttributes);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String)
- */
- public Object readResult(Result result, String qualifier) throws IOException {
- byte[] columnQualifier =
- ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
- return column.readResult(result, columnQualifier);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResults(org.apache.hadoop.hbase.client.Result,
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
- */
- public Map readResults(Result result,
- KeyConverter keyConverter) throws IOException {
- return column.readResults(result, columnPrefixBytes, keyConverter);
- }
-
- /*
- * (non-Javadoc)
- *
- * @see
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
- * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
- * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
- */
- public NavigableMap>
- readResultsWithTimestamps(Result result, KeyConverter keyConverter)
- throws IOException {
- return column.readResultsWithTimestamps(result, columnPrefixBytes,
- keyConverter);
- }
-
- /**
- * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is
- * no match. The following holds true: {@code columnFor(x) == columnFor(y)} if
- * and only if {@code x.equals(y)} or {@code (x == y == null)}
- *
- * @param columnPrefix Name of the column to retrieve
- * @return the corresponding {@link FlowRunColumnPrefix} or null
- */
- public static final FlowRunColumnPrefix columnFor(String columnPrefix) {
-
- // Match column based on value, assume column family matches.
- for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) {
- // Find a match based only on name.
- if (frcp.getColumnPrefix().equals(columnPrefix)) {
- return frcp;
- }
- }
-
- // Default to null
- return null;
- }
-
- @Override
- public ValueConverter getValueConverter() {
- return column.getValueConverter();
- }
-
- /**
- * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is
- * no match. The following holds true:
- * {@code columnFor(a,x) == columnFor(b,y)} if and only if
- * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
- *
- * @param columnFamily The columnFamily for which to retrieve the column.
- * @param columnPrefix Name of the column to retrieve
- * @return the corresponding {@link FlowRunColumnPrefix} or null if both
- * arguments don't match.
- */
- public static final FlowRunColumnPrefix columnFor(
- FlowRunColumnFamily columnFamily, String columnPrefix) {
-
- // TODO: needs unit test to confirm and need to update javadoc to explain
- // null prefix case.
-
- for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) {
- // Find a match based column family and on name.
- if (frcp.columnFamily.equals(columnFamily)
- && (((columnPrefix == null) && (frcp.getColumnPrefix() == null)) ||
- (frcp.getColumnPrefix().equals(columnPrefix)))) {
- return frcp;
- }
- }
-
- // Default to null
- return null;
- }
-}
diff --git hadoop-yarn-project/pom.xml hadoop-yarn-project/pom.xml
index eddec09..2820f9f 100644
--- hadoop-yarn-project/pom.xml
+++ hadoop-yarn-project/pom.xml
@@ -77,7 +77,7 @@
org.apache.hadoop
- hadoop-yarn-server-timelineservice-hbase
+ hadoop-yarn-server-timelineservice-hbase-client