diff --git hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml index a77dd20..d816ec3 100644 --- hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml +++ hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml @@ -213,7 +213,11 @@ - hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/target/lib + hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/target/lib + share/hadoop/${hadoop.component}/timelineservice/lib + + + hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/target/lib share/hadoop/${hadoop.component}/timelineservice/lib @@ -221,7 +225,9 @@ org.apache.hadoop:hadoop-yarn-server-timelineservice - org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase + org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-client + org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-common + org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-server share/hadoop/${hadoop.component}/timelineservice diff --git hadoop-project/pom.xml hadoop-project/pom.xml index dd8465a..99af3b8 100644 --- hadoop-project/pom.xml +++ hadoop-project/pom.xml @@ -397,7 +397,19 @@ org.apache.hadoop - hadoop-yarn-server-timelineservice-hbase + hadoop-yarn-server-timelineservice-hbase-client + ${project.version} + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase-common + ${project.version} + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase-server ${project.version} 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 f36897b..d9f992d 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 @@ -60,7 +60,31 @@ org.apache.hadoop - hadoop-yarn-server-timelineservice-hbase + hadoop-yarn-server-timelineservice-hbase-client + test + + + org.apache.hadoop + hadoop-common + + + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase-common + test + + + org.apache.hadoop + hadoop-common + + + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase-server test 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 b2029ca..33d8dcd 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 @@ -50,7 +50,7 @@ import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; 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.junit.AfterClass; import org.junit.BeforeClass; @@ -70,7 +70,7 @@ extends AbstractTimelineReaderHBaseTestBase { private static long ts = System.currentTimeMillis(); private static long dayTs = - HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts); + HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(ts); private static String doAsUser = "remoteuser"; @BeforeClass @@ -371,7 +371,7 @@ static void writeApplicationEntities(HBaseTimelineWriterImpl hbi, BuilderUtils.newApplicationId(timestamp, count++); ApplicationEntity appEntity = new ApplicationEntity(); appEntity.setId( - HBaseTimelineStorageUtils.convertApplicationIdToString(appId)); + HBaseTimelineSchemaUtils.convertApplicationIdToString(appId)); appEntity.setCreatedTime(timestamp); TimelineEvent created = new TimelineEvent(); @@ -929,7 +929,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:" + getServerPort() + "/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 111008a..bc33427 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 @@ -68,10 +68,11 @@ 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.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.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.Separator; import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter; @@ -173,7 +174,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); @@ -308,7 +309,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()); @@ -319,24 +320,24 @@ 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 = ColumnRWHelper.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 = ColumnRWHelper.readResult( + result, ApplicationColumnPrefix.IS_RELATED_TO, + isRelatedToEntry.getKey()); String compoundValue = isRelatedToValue.toString(); // id7?id9?id6 Set isRelatedToValues = @@ -351,9 +352,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 = ColumnRWHelper.readResult(result, + ApplicationColumnPrefix.RELATES_TO, relatesToEntry.getKey()) + .toString(); // id3?id4?id5 Set relatesToValues = new HashSet(Separator.VALUES.splitEncoded(compoundValue)); @@ -366,14 +367,13 @@ public void testWriteApplicationToHBase() throws Exception { KeyConverter stringKeyConverter = new StringKeyConverter(); // Configuration - Map configColumns = - ApplicationColumnPrefix.CONFIG - .readResults(result, stringKeyConverter); + Map configColumns = ColumnRWHelper.readResults( + result, ApplicationColumnPrefix.CONFIG, stringKeyConverter); assertEquals(conf, configColumns); NavigableMap> metricsResult = - ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result, - stringKeyConverter); + ColumnRWHelper.readResultsWithTimestamps( + result, ApplicationColumnPrefix.METRIC, stringKeyConverter); NavigableMap metricMap = metricsResult.get(m1.getId()); matchMetrics(metricValues, metricMap); @@ -500,7 +500,7 @@ public void testEvents() throws IOException { event.addInfo(expKey, expVal); final TimelineEntity entity = new ApplicationEntity(); - entity.setId(HBaseTimelineStorageUtils.convertApplicationIdToString( + entity.setId(HBaseTimelineSchemaUtils.convertApplicationIdToString( ApplicationId.newInstance(0, 1))); entity.addEvent(event); @@ -531,7 +531,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); @@ -541,8 +541,8 @@ public void testEvents() throws IOException { appName)); Map eventsResult = - ApplicationColumnPrefix.EVENT.readResults(result, - new EventColumnNameConverter()); + ColumnRWHelper.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 5e08999..90a6959 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 @@ -62,9 +62,10 @@ 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.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.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.Separator; import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter; @@ -73,12 +74,12 @@ 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.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn; import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix; import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTableRW; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -208,7 +209,7 @@ public void testWriteEntityToHBase() throws Exception { String flow = "some_flow_name"; String flowVersion = "AB7822C10F1111"; long runid = 1002345678919L; - String appName = HBaseTimelineStorageUtils.convertApplicationIdToString( + String appName = HBaseTimelineSchemaUtils.convertApplicationIdToString( ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1) ); hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, @@ -224,7 +225,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; @@ -238,26 +239,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 = ColumnRWHelper.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 = ColumnRWHelper.readResult(result, + EntityColumnPrefix.IS_RELATED_TO, isRelatedToEntry.getKey()); String compoundValue = isRelatedToValue.toString(); // id7?id9?id6 Set isRelatedToValues = @@ -273,8 +275,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 = ColumnRWHelper.readResult(result, + EntityColumnPrefix.RELATES_TO, relatesToEntry.getKey()) + .toString(); // id3?id4?id5 Set relatesToValues = new HashSet( @@ -287,13 +290,13 @@ public void testWriteEntityToHBase() throws Exception { } // Configuration - Map configColumns = - EntityColumnPrefix.CONFIG.readResults(result, stringKeyConverter); + Map configColumns = ColumnRWHelper.readResults( + result, EntityColumnPrefix.CONFIG, stringKeyConverter); assertEquals(conf, configColumns); NavigableMap> metricsResult = - EntityColumnPrefix.METRIC.readResultsWithTimestamps(result, - stringKeyConverter); + ColumnRWHelper.readResultsWithTimestamps( + result, EntityColumnPrefix.METRIC, stringKeyConverter); NavigableMap metricMap = metricsResult.get(m1.getId()); matchMetrics(metricValues, metricMap); @@ -386,14 +389,14 @@ private void verifySubApplicationTableEntities(String cluster, String user, Set metrics, Long cTime, TimelineMetric m1) throws IOException { Scan s = new Scan(); - // read from SubApplicationTable + // read from SubApplicationTableRW byte[] startRow = new SubApplicationRowKeyPrefix(cluster, subAppUser, null, null, null, null).getRowKeyPrefix(); s.setStartRow(startRow); s.setMaxVersions(Integer.MAX_VALUE); Connection conn = ConnectionFactory.createConnection(c1); ResultScanner scanner = - new SubApplicationTable().getResultScanner(c1, conn, s); + new SubApplicationTableRW().getResultScanner(c1, conn, s); int rowCount = 0; int colCount = 0; @@ -407,25 +410,28 @@ private void verifySubApplicationTableEntities(String cluster, String user, user, entity)); // check info column family - String id1 = SubApplicationColumn.ID.readResult(result).toString(); + String id1 = ColumnRWHelper.readResult(result, SubApplicationColumn.ID) + .toString(); assertEquals(id, id1); - String type1 = SubApplicationColumn.TYPE.readResult(result).toString(); + String type1 = ColumnRWHelper.readResult(result, + SubApplicationColumn.TYPE).toString(); assertEquals(type, type1); - Long cTime1 = - (Long) SubApplicationColumn.CREATED_TIME.readResult(result); + Long cTime1 = (Long) ColumnRWHelper.readResult(result, + SubApplicationColumn.CREATED_TIME); assertEquals(cTime1, cTime); - Map infoColumns = SubApplicationColumnPrefix.INFO - .readResults(result, new StringKeyConverter()); + Map infoColumns = ColumnRWHelper.readResults( + result, SubApplicationColumnPrefix.INFO, new StringKeyConverter()); assertEquals(infoMap, infoColumns); // Remember isRelatedTo is of type Map> for (Map.Entry> isRelatedToEntry : isRelatedTo .entrySet()) { - Object isRelatedToValue = SubApplicationColumnPrefix.IS_RELATED_TO - .readResult(result, isRelatedToEntry.getKey()); + Object isRelatedToValue = ColumnRWHelper.readResult( + result, SubApplicationColumnPrefix.IS_RELATED_TO, + isRelatedToEntry.getKey()); String compoundValue = isRelatedToValue.toString(); // id7?id9?id6 Set isRelatedToValues = @@ -440,8 +446,9 @@ private void verifySubApplicationTableEntities(String cluster, String user, // RelatesTo for (Map.Entry> relatesToEntry : relatesTo .entrySet()) { - String compoundValue = SubApplicationColumnPrefix.RELATES_TO - .readResult(result, relatesToEntry.getKey()).toString(); + String compoundValue = ColumnRWHelper.readResult(result, + SubApplicationColumnPrefix.RELATES_TO, relatesToEntry.getKey()) + .toString(); // id3?id4?id5 Set relatesToValues = new HashSet(Separator.VALUES.splitEncoded(compoundValue)); @@ -453,13 +460,13 @@ private void verifySubApplicationTableEntities(String cluster, String user, } // Configuration - Map configColumns = SubApplicationColumnPrefix.CONFIG - .readResults(result, stringKeyConverter); + Map configColumns = ColumnRWHelper.readResults( + result, SubApplicationColumnPrefix.CONFIG, stringKeyConverter); assertEquals(conf, configColumns); NavigableMap> metricsResult = - SubApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result, - stringKeyConverter); + ColumnRWHelper.readResultsWithTimestamps( + result, SubApplicationColumnPrefix.METRIC, stringKeyConverter); NavigableMap metricMap = metricsResult.get(m1.getId()); matchMetrics(metricValues, metricMap); @@ -511,7 +518,7 @@ public void testEventsWithEmptyInfo() throws IOException { String flow = "other_flow_name"; String flowVersion = "1111F01C2287BA"; long runid = 1009876543218L; - String appName = HBaseTimelineStorageUtils.convertApplicationIdToString( + String appName = HBaseTimelineSchemaUtils.convertApplicationIdToString( ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1)); byte[] startRow = new EntityRowKeyPrefix(cluster, user, flow, runid, appName) @@ -525,7 +532,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) { @@ -538,8 +545,8 @@ public void testEventsWithEmptyInfo() throws IOException { entity)); Map eventsResult = - EntityColumnPrefix.EVENT.readResults(result, - new EventColumnNameConverter()); + ColumnRWHelper.readResults(result, + EntityColumnPrefix.EVENT, new EventColumnNameConverter()); // there should be only one event assertEquals(1, eventsResult.size()); for (Map.Entry e : eventsResult.entrySet()) { @@ -604,7 +611,7 @@ public void testEventsEscapeTs() throws IOException { final TimelineEntity entity = new ApplicationEntity(); entity.setId( - HBaseTimelineStorageUtils.convertApplicationIdToString( + HBaseTimelineSchemaUtils.convertApplicationIdToString( ApplicationId.newInstance(0, 1))); entity.addEvent(event); 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..f838178 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,24 @@ 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 +92,22 @@ 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 +117,22 @@ 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 4bf221e..645b7d5 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 @@ -52,9 +52,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; @@ -161,8 +161,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); @@ -178,7 +178,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); @@ -292,8 +292,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) { @@ -309,7 +309,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()); @@ -401,7 +401,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()); @@ -442,8 +442,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) { @@ -456,7 +456,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 1ad02e1..622b0eb 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 @@ -62,9 +62,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; @@ -91,8 +91,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(); @@ -106,9 +106,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 @@ -116,8 +116,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 @@ -224,8 +224,10 @@ 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(); @@ -380,8 +382,10 @@ 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; @@ -525,8 +529,10 @@ 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; @@ -810,8 +816,10 @@ 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 0ef8260..31be285 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 @@ -54,9 +54,9 @@ import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; 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.HBaseTimelineServerUtils; import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; import org.junit.AfterClass; @@ -107,8 +107,10 @@ 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); @@ -156,8 +158,10 @@ 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"; @@ -324,10 +328,12 @@ 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 @@ -352,8 +358,10 @@ 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; @@ -420,7 +428,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cell1Ts, Bytes.toBytes(cellValue1), tagByteArray); currentColumnCells.add(c1); @@ -430,7 +438,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cell2Ts, Bytes.toBytes(cellValue2), tagByteArray); currentColumnCells.add(c2); @@ -440,7 +448,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cell3Ts, Bytes.toBytes(cellValue3), tagByteArray); currentColumnCells.add(c3); @@ -450,7 +458,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cell4Ts, Bytes.toBytes(cellValue4), tagByteArray); currentColumnCells.add(c4); @@ -520,7 +528,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray); currentColumnCells.add(c1); cellTsFinal++; @@ -534,7 +542,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray); currentColumnCells.add(c1); cellTsNotFinal++; @@ -611,7 +619,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray); currentColumnCells.add(c1); cellTsFinal++; @@ -625,7 +633,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cellTsFinalNotExpire, Bytes.toBytes(cellValueFinal), tagByteArray); currentColumnCells.add(c1); cellTsFinalNotExpire++; @@ -639,7 +647,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray); currentColumnCells.add(c1); cellTsNotFinal++; @@ -696,7 +704,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, 120L, Bytes.toBytes(cellValue1), tagByteArray); currentColumnCells.add(c1); @@ -707,7 +715,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, 130L, Bytes.toBytes(cellValue2), tagByteArray); currentColumnCells.add(c2); List cells = fs.processSummationMajorCompaction(currentColumnCells, @@ -754,7 +762,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 = HBaseTimelineServerUtils.createNewCell(aRowKey, aFamily, aQualifier, 120L, Bytes.toBytes(1110L), tagByteArray); currentColumnCells.add(c1); @@ -792,7 +800,7 @@ public void testProcessSummationOneCell() throws IOException { SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR); - Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + Cell c1 = HBaseTimelineServerUtils.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..a1db497 --- /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,219 @@ + + + + + hadoop-yarn-server-timelineservice-hbase + org.apache.hadoop + 3.2.0-SNAPSHOT + + 4.0.0 + hadoop-yarn-server-timelineservice-hbase-client + Apache Hadoop YARN TimelineService HBase Client + + + + ${project.parent.parent.parent.basedir} + + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase-common + + + + commons-logging + commons-logging + + + + commons-lang + commons-lang + + + + commons-cli + commons-cli + + + + com.google.guava + guava + + + + org.apache.hadoop + hadoop-annotations + provided + + + + org.apache.hadoop + hadoop-common + provided + + + + + org.apache.hadoop + hadoop-common + test-jar + test + + + + org.apache.hadoop + hadoop-yarn-api + provided + + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + provided + + + + 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 + provided + + + 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 + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + package + + copy-dependencies + + + runtime + org.slf4j,org.apache.hadoop,com.github.stephenc.findbugs + ${project.build.directory}/lib + + + + + + + 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 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/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 index 8b46d32..2b98eec 100644 --- 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 @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; @@ -98,7 +99,7 @@ private static CompareOp getHBaseCompareOp( * @param filter * @return a {@link QualifierFilter} object */ - private static Filter createHBaseColQualPrefixFilter( + private static > Filter createHBaseColQualPrefixFilter( ColumnPrefix colPrefix, TimelinePrefixFilter filter) { return new QualifierFilter(getHBaseCompareOp(filter.getCompareOp()), new BinaryPrefixComparator( @@ -114,8 +115,8 @@ private static CompareOp getHBaseCompareOp( * @param columnPrefix column prefix. * @return a column qualifier filter. */ - public static Filter createHBaseQualifierFilter(CompareOp compareOp, - ColumnPrefix columnPrefix) { + public static > Filter createHBaseQualifierFilter( + CompareOp compareOp, ColumnPrefix columnPrefix) { return new QualifierFilter(compareOp, new BinaryPrefixComparator( columnPrefix.getColumnPrefixBytes(""))); @@ -133,7 +134,8 @@ private static CompareOp getHBaseCompareOp( * @return a filter list. * @throws IOException if any problem occurs while creating the filters. */ - public static Filter createFilterForConfsOrMetricsToRetrieve( + public static > Filter + createFilterForConfsOrMetricsToRetrieve( TimelineFilterList confsOrMetricToRetrieve, ColumnFamily columnFamily, ColumnPrefix columnPrefix) throws IOException { Filter familyFilter = new FamilyFilter(CompareOp.EQUAL, @@ -164,8 +166,9 @@ private static CompareOp getHBaseCompareOp( * @return 2 single column value filters wrapped in a filter list. * @throws IOException if any problem is encountered while encoding value. */ - public static FilterList createSingleColValueFiltersByRange( - Column column, Object startValue, Object endValue) throws IOException { + public static > FilterList + createSingleColValueFiltersByRange(Column column, + Object startValue, Object endValue) throws IOException { FilterList list = new FilterList(); Filter singleColValFilterStart = createHBaseSingleColValueFilter( column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), @@ -190,8 +193,9 @@ private static CompareOp getHBaseCompareOp( * @return a SingleColumnValue Filter * @throws IOException if any exception. */ - public static Filter createHBaseSingleColValueFilter(Column column, - Object value, CompareOp op) throws IOException { + public static > Filter + createHBaseSingleColValueFilter(Column column, + Object value, CompareOp op) throws IOException { Filter singleColValFilter = createHBaseSingleColValueFilter( column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), column.getValueConverter().encodeValue(value), op, true); @@ -263,7 +267,8 @@ private static SingleColumnValueFilter createHBaseSingleColValueFilter( * @return A {@link FilterList} object. * @throws IOException if any problem occurs while creating the filter list. */ - public static FilterList createHBaseFilterList(ColumnPrefix colPrefix, + public static > FilterList createHBaseFilterList( + ColumnPrefix colPrefix, TimelineFilterList filterList) throws IOException { FilterList list = new FilterList(getHBaseOperator(filterList.getOperator())); 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..027505b --- /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,611 @@ +/** + * 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.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.security.UserGroupInformation; +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.collector.TimelineCollectorContext; +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.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +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; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTableRW; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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 Logger LOG = LoggerFactory + .getLogger(HBaseTimelineWriterImpl.class); + + private Connection conn; + private TypedBufferedMutator entityTable; + private TypedBufferedMutator appToFlowTable; + private TypedBufferedMutator applicationTable; + private TypedBufferedMutator flowActivityTable; + private TypedBufferedMutator flowRunTable; + private TypedBufferedMutator subApplicationTable; + + /** + * 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(); + + private enum Tables { + APPLICATION_TABLE, ENTITY_TABLE, SUBAPPLICATION_TABLE + }; + + public HBaseTimelineWriterImpl() { + super(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); + subApplicationTable = + new SubApplicationTableRW().getTableMutator(hbaseConf, conn); + + UserGroupInformation ugi = UserGroupInformation.isSecurityEnabled() ? + UserGroupInformation.getLoginUser() : + UserGroupInformation.getCurrentUser(); + LOG.info("Initialized HBaseTimelineWriterImpl UGI to " + ugi); + } + + /** + * Stores the entire information in TimelineEntities to the timeline store. + */ + @Override + public TimelineWriteResponse write(TimelineCollectorContext context, + TimelineEntities data, UserGroupInformation callerUgi) + throws IOException { + + TimelineWriteResponse putStatus = new TimelineWriteResponse(); + + String clusterId = context.getClusterId(); + String userId = context.getUserId(); + String flowName = context.getFlowName(); + String flowVersion = context.getFlowVersion(); + long flowRunId = context.getFlowRunId(); + String appId = context.getAppId(); + String subApplicationUser = callerUgi.getShortUserName(); + + // 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(); + store(rowKey, te, flowVersion, Tables.APPLICATION_TABLE); + } else { + EntityRowKey entityRowKey = + new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, + te.getType(), te.getIdPrefix(), te.getId()); + rowKey = entityRowKey.getRowKey(); + store(rowKey, te, flowVersion, Tables.ENTITY_TABLE); + } + + if (!isApplication && !userId.equals(subApplicationUser)) { + SubApplicationRowKey subApplicationRowKey = + new SubApplicationRowKey(subApplicationUser, clusterId, + te.getType(), te.getIdPrefix(), te.getId(), userId); + rowKey = subApplicationRowKey.getRowKey(); + store(rowKey, te, flowVersion, Tables.SUBAPPLICATION_TABLE); + } + + 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(); + ColumnRWHelper.store(rowKey, appToFlowTable, + AppToFlowColumnPrefix.FLOW_NAME, clusterId, null, flowName); + ColumnRWHelper.store(rowKey, appToFlowTable, + AppToFlowColumnPrefix.FLOW_RUN_ID, clusterId, null, flowRunId); + ColumnRWHelper.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()); + ColumnRWHelper.store(flowActivityRowKeyBytes, flowActivityTable, + FlowActivityColumnPrefix.RUN_ID, 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(); + ColumnRWHelper.store(rowKey, flowRunTable, FlowRunColumn.MIN_START_TIME, + 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()); + ColumnRWHelper.store(rowKey, flowActivityTable, + FlowActivityColumnPrefix.RUN_ID, 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); + 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 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(); + ColumnRWHelper.store(rowKey, flowRunTable, FlowRunColumnPrefix.METRIC, + metricColumnQualifier, timestamp, timeseriesEntry.getValue(), + attributes); + } + } + } + + /** + * Stores the Relations from the {@linkplain TimelineEntity} object. + */ + private > void storeRelations(byte[] rowKey, + Map> connectedEntities, ColumnPrefix columnPrefix, + TypedBufferedMutator table) throws IOException { + if (connectedEntities != null) { + for (Map.Entry> connectedEntity : connectedEntities + .entrySet()) { + // id3?id4?id5 + String compoundValue = + Separator.VALUES.joinEncoded(connectedEntity.getValue()); + ColumnRWHelper.store(rowKey, table, columnPrefix, + stringKeyConverter.encode(connectedEntity.getKey()), + null, compoundValue); + } + } + } + + /** + * Stores information from the {@linkplain TimelineEntity} object. + */ + private void store(byte[] rowKey, TimelineEntity te, + String flowVersion, + Tables table) throws IOException { + switch (table) { + case APPLICATION_TABLE: + 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); + storeInfo(rowKey, te.getInfo(), flowVersion, ApplicationColumnPrefix.INFO, + applicationTable); + storeMetrics(rowKey, te.getMetrics(), ApplicationColumnPrefix.METRIC, + applicationTable); + storeEvents(rowKey, te.getEvents(), ApplicationColumnPrefix.EVENT, + applicationTable); + storeConfig(rowKey, te.getConfigs(), ApplicationColumnPrefix.CONFIG, + applicationTable); + storeRelations(rowKey, te.getIsRelatedToEntities(), + ApplicationColumnPrefix.IS_RELATED_TO, applicationTable); + storeRelations(rowKey, te.getRelatesToEntities(), + ApplicationColumnPrefix.RELATES_TO, applicationTable); + break; + case ENTITY_TABLE: + 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); + storeInfo(rowKey, te.getInfo(), flowVersion, EntityColumnPrefix.INFO, + entityTable); + storeMetrics(rowKey, te.getMetrics(), EntityColumnPrefix.METRIC, + entityTable); + storeEvents(rowKey, te.getEvents(), EntityColumnPrefix.EVENT, + entityTable); + storeConfig(rowKey, te.getConfigs(), EntityColumnPrefix.CONFIG, + entityTable); + storeRelations(rowKey, te.getIsRelatedToEntities(), + EntityColumnPrefix.IS_RELATED_TO, entityTable); + storeRelations(rowKey, te.getRelatesToEntities(), + EntityColumnPrefix.RELATES_TO, entityTable); + break; + case SUBAPPLICATION_TABLE: + ColumnRWHelper.store(rowKey, subApplicationTable, SubApplicationColumn.ID, + null, te.getId()); + ColumnRWHelper.store(rowKey, subApplicationTable, + SubApplicationColumn.TYPE, null, te.getType()); + ColumnRWHelper.store(rowKey, subApplicationTable, + SubApplicationColumn.CREATED_TIME, null, te.getCreatedTime()); + ColumnRWHelper.store(rowKey, subApplicationTable, + SubApplicationColumn.FLOW_VERSION, null, flowVersion); + storeInfo(rowKey, te.getInfo(), flowVersion, + SubApplicationColumnPrefix.INFO, subApplicationTable); + storeMetrics(rowKey, te.getMetrics(), SubApplicationColumnPrefix.METRIC, + subApplicationTable); + storeEvents(rowKey, te.getEvents(), SubApplicationColumnPrefix.EVENT, + subApplicationTable); + storeConfig(rowKey, te.getConfigs(), SubApplicationColumnPrefix.CONFIG, + subApplicationTable); + storeRelations(rowKey, te.getIsRelatedToEntities(), + SubApplicationColumnPrefix.IS_RELATED_TO, subApplicationTable); + storeRelations(rowKey, te.getRelatesToEntities(), + SubApplicationColumnPrefix.RELATES_TO, subApplicationTable); + break; + default: + LOG.info("Invalid table name provided."); + break; + } + } + + /** + * stores the info information from {@linkplain TimelineEntity}. + */ + private > void storeInfo(byte[] rowKey, + Map info, String flowVersion, + ColumnPrefix columnPrefix, TypedBufferedMutator table) + throws IOException { + if (info != null) { + for (Map.Entry entry : info.entrySet()) { + ColumnRWHelper.store(rowKey, table, columnPrefix, + stringKeyConverter.encode(entry.getKey()), null, entry.getValue()); + } + } + } + + /** + * stores the config information from {@linkplain TimelineEntity}. + */ + private > void storeConfig( + byte[] rowKey, Map config, + ColumnPrefix columnPrefix, TypedBufferedMutator table) + throws IOException { + if (config != null) { + for (Map.Entry entry : config.entrySet()) { + byte[] configKey = stringKeyConverter.encode(entry.getKey()); + ColumnRWHelper.store(rowKey, table, columnPrefix, configKey, + null, entry.getValue()); + } + } + } + + /** + * stores the {@linkplain TimelineMetric} information from the + * {@linkplain TimelineEvent} object. + */ + private > void storeMetrics( + byte[] rowKey, Set metrics, + ColumnPrefix columnPrefix, TypedBufferedMutator table) + 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(); + ColumnRWHelper.store(rowKey, table, columnPrefix, + metricColumnQualifier, timestamp, timeseriesEntry.getValue()); + } + } + } + } + + /** + * Stores the events from the {@linkplain TimelineEvent} object. + */ + private > void storeEvents( + byte[] rowKey, Set events, + ColumnPrefix columnPrefix, TypedBufferedMutator table) + 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(); + ColumnRWHelper.store(rowKey, table, columnPrefix, + columnQualifierBytes, null, Separator.EMPTY_BYTES); + } else { + for (Map.Entry info : eventInfo.entrySet()) { + // eventId=infoKey + byte[] columnQualifierBytes = + new EventColumnName(eventId, eventTimestamp, info.getKey()) + .getColumnQualifier(); + ColumnRWHelper.store(rowKey, table, columnPrefix, + 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(); + subApplicationTable.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 (subApplicationTable != null) { + subApplicationTable.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 c9f7cec..e9e4770 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 @@ -37,13 +37,13 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.yarn.conf.YarnConfiguration; -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.subapplication.SubApplicationTable; +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 org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTableRW; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; @@ -94,26 +94,27 @@ 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); } // Grab the entity metrics TTL String entityTableMetricsTTL = commandLine.getOptionValue( ENTITY_METRICS_TTL_OPTION_SHORT); if (StringUtils.isNotBlank(entityTableMetricsTTL)) { int entityMetricsTTL = Integer.parseInt(entityTableMetricsTTL); - new EntityTable().setMetricsTTL(entityMetricsTTL, hbaseConf); + new EntityTableRW().setMetricsTTL(entityMetricsTTL, 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); } // Grab the application metrics TTL @@ -121,14 +122,14 @@ public static void main(String[] args) throws Exception { APP_METRICS_TTL_OPTION_SHORT); if (StringUtils.isNotBlank(applicationTableMetricsTTL)) { int appMetricsTTL = Integer.parseInt(applicationTableMetricsTTL); - new ApplicationTable().setMetricsTTL(appMetricsTTL, hbaseConf); + new ApplicationTableRW().setMetricsTTL(appMetricsTTL, hbaseConf); } // Grab the subApplicationTableName argument String subApplicationTableName = commandLine.getOptionValue( SUB_APP_TABLE_NAME_SHORT); if (StringUtils.isNotBlank(subApplicationTableName)) { - hbaseConf.set(SubApplicationTable.TABLE_NAME_CONF_NAME, + hbaseConf.set(SubApplicationTableRW.TABLE_NAME_CONF_NAME, subApplicationTableName); } // Grab the subApplication metrics TTL @@ -136,7 +137,7 @@ public static void main(String[] args) throws Exception { .getOptionValue(SUB_APP_METRICS_TTL_OPTION_SHORT); if (StringUtils.isNotBlank(subApplicationTableMetricsTTL)) { int subAppMetricsTTL = Integer.parseInt(subApplicationTableMetricsTTL); - new SubApplicationTable().setMetricsTTL(subAppMetricsTTL, hbaseConf); + new SubApplicationTableRW().setMetricsTTL(subAppMetricsTTL, hbaseConf); } // create all table schemas in hbase @@ -303,7 +304,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()); @@ -312,7 +313,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()); @@ -321,7 +322,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()); @@ -330,7 +331,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()); @@ -339,7 +340,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()); @@ -348,7 +349,7 @@ public static void createAllTables(Configuration hbaseConf, } } try { - new SubApplicationTable().createTable(admin, hbaseConf); + new SubApplicationTableRW().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/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTableRW.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 new file mode 100644 index 0000000..808994e --- /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/application/ApplicationTableRW.java @@ -0,0 +1,137 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +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.BaseTableRW; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Create, read and write to the Application Table. + */ +public class ApplicationTableRW extends BaseTableRW { + /** application prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "application"; + + /** config param name that specifies the application table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** + * config param name that specifies the TTL for metrics column family in + * application table. + */ + private static final String METRICS_TTL_CONF_NAME = PREFIX + + ".table.metrics.ttl"; + + /** + * config param name that specifies max-versions for metrics column family in + * entity table. + */ + private static final String METRICS_MAX_VERSIONS = + PREFIX + ".table.metrics.max-versions"; + + /** default value for application table name. */ + private static final String DEFAULT_TABLE_NAME = + "timelineservice.application"; + + /** default TTL is 30 days for metrics timeseries. */ + private static final int DEFAULT_METRICS_TTL = 2592000; + + /** default max number of versions. */ + private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; + + private static final Logger LOG = + LoggerFactory.getLogger(ApplicationTableRW.class); + + public ApplicationTableRW() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW# + * createTable(org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor applicationTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(ApplicationColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + applicationTableDescp.addFamily(infoCF); + + HColumnDescriptor configCF = + new HColumnDescriptor(ApplicationColumnFamily.CONFIGS.getBytes()); + configCF.setBloomFilterType(BloomType.ROWCOL); + configCF.setBlockCacheEnabled(true); + applicationTableDescp.addFamily(configCF); + + HColumnDescriptor metricsCF = + new HColumnDescriptor(ApplicationColumnFamily.METRICS.getBytes()); + applicationTableDescp.addFamily(metricsCF); + metricsCF.setBlockCacheEnabled(true); + // always keep 1 version (the latest) + metricsCF.setMinVersions(1); + metricsCF.setMaxVersions( + hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); + metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, + DEFAULT_METRICS_TTL)); + applicationTableDescp.setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + applicationTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(applicationTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } + + /** + * @param metricsTTL time to live parameter for the metrics in this table. + * @param hbaseConf configuration in which to set the metrics TTL config + * variable. + */ + public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { + hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); + } + +} 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/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTableRW.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 new file mode 100644 index 0000000..6460203 --- /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/apptoflow/AppToFlowTableRW.java @@ -0,0 +1,92 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +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.BaseTableRW; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * Create, read and write to the AppToFlow Table. + */ +public class AppToFlowTableRW extends BaseTableRW { + /** app_flow prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "app-flow"; + + /** config param name that specifies the app_flow table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for app_flow table name. */ + private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow"; + + private static final Logger LOG = + LoggerFactory.getLogger(AppToFlowTableRW.class); + + public AppToFlowTableRW() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW# + * createTable(org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor appToFlowTableDescp = new HTableDescriptor(table); + HColumnDescriptor mappCF = + new HColumnDescriptor(AppToFlowColumnFamily.MAPPING.getBytes()); + mappCF.setBloomFilterType(BloomType.ROWCOL); + appToFlowTableDescp.addFamily(mappCF); + + appToFlowTableDescp + .setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + appToFlowTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(appToFlowTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + 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/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 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/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..12ebce4 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; } @@ -82,7 +82,7 @@ protected BaseTable(String tableNameConfName, String defaultTableName) { // This is how service initialization should hang on to this variable, with // the proper type TypedBufferedMutator table = - new BufferedMutatorDelegator(bufferedMutator); + new TypedBufferedMutator(bufferedMutator); return table; } 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..a8e5149 --- /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,487 @@ +/** + * 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.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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; + +/** + * A set of utility functions that read or read to a column. + * This class is meant to be used only by explicit Columns, + * and not directly to write by clients. + */ +public final class ColumnRWHelper { + private static final Logger LOG = + LoggerFactory.getLogger(ColumnHelper.class); + + private ColumnRWHelper() { + } + + /** + * Figures out the cell timestamp used in the Put For storing. + * Will supplement the timestamp if required. Typically done for flow run + * table.If we supplement the timestamp, we 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, boolean supplementTs, Attribute[] attributes) { + if (timestamp == null) { + timestamp = System.currentTimeMillis(); + } + if (!supplementTs) { + return timestamp; + } else { + 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; + } + + /** + * 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, + column.supplementCellTimestamp(), inputValue, + column.getValueConverter(), + column.getCombinedAttrsWithAggr(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, + boolean supplementTs, Object inputValue, ValueConverter converter, + Attribute... attributes) throws IOException { + if ((rowKey == null) || (columnQualifier == null) || (inputValue == null)) { + return; + } + Put p = new Put(rowKey); + timestamp = getPutTimestamp(timestamp, supplementTs, 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); + } + + /** + * 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()); + } + + /** + * 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 columnPrefix column prefix to read from + * @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.getColumnPrefixInBytes(), qualifier); + + return 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 readResults(result, + columnPrefix.getColumnFamilyBytes(), + columnPrefix.getColumnPrefixInBytes(), + 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 readResultsWithTimestamps(result, + columnPrefix.getColumnFamilyBytes(), + columnPrefix.getColumnPrefixInBytes(), + keyConverter, columnPrefix.getValueConverter(), + columnPrefix.supplementCellTimeStamp()); + } + + /** + * @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 static NavigableMap> + readResultsWithTimestamps(Result result, byte[] columnFamilyBytes, + byte[] columnPrefixBytes, KeyConverter keyConverter, + ValueConverter valueConverter, boolean supplementTs) + 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) valueConverter.decodeValue(cell.getValue()); + Long ts = supplementTs ? TimestampGenerator. + getTruncatedTimestamp(cell.getKey()) : cell.getKey(); + cellResults.put(ts, 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 static Map readResults(Result result, + byte[] columnFamilyBytes, byte[] columnPrefixBytes, + KeyConverter keyConverter, ValueConverter valueConverter) + 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 = valueConverter.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; + } + + /** + * 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.getCombinedAttrsWithAggr(attributes); + + store(rowKey, tableMutator, columnPrefix.getColumnFamilyBytes(), + columnQualifier, timestamp, columnPrefix.supplementCellTimeStamp(), + 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.getCombinedAttrsWithAggr(attributes); + + store(rowKey, tableMutator, columnPrefix.getColumnFamilyBytes(), + columnQualifier, timestamp, columnPrefix.supplementCellTimeStamp(), + inputValue, columnPrefix.getValueConverter(), combinedAttributes); + } +} 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..f4cd6fb --- /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,121 @@ +/** + * 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.net.MalformedURLException; +import java.net.URL; +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Query; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A bunch of utility functions used in HBase TimelineService backend. + */ +public final class HBaseTimelineStorageUtils { + private static final Logger LOG = + LoggerFactory.getLogger(HBaseTimelineStorageUtils.class); + + private HBaseTimelineStorageUtils() { + } + + + /** + * @param conf YARN configuration. Used to see if there is an explicit config + * pointing to the HBase config file to read. It should not be null + * or a NullPointerException will be thrown. + * @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 { + if (conf == null) { + throw new NullPointerException(); + } + + Configuration hbaseConf; + String timelineServiceHBaseConfFileURL = + conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE); + if (timelineServiceHBaseConfFileURL != null + && timelineServiceHBaseConfFileURL.length() > 0) { + LOG.info("Using hbase configuration at " + + timelineServiceHBaseConfFileURL); + // 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; + } + + public static void setMetricsTimeRange(Query query, byte[] metricsCf, + long tsBegin, long tsEnd) { + if (tsBegin != 0 || tsEnd != Long.MAX_VALUE) { + query.setColumnFamilyTimeRange(metricsCf, + tsBegin, ((tsEnd == Long.MAX_VALUE) ? Long.MAX_VALUE : (tsEnd + 1))); + } + } +} 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/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/TypedBufferedMutator.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/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/TypedBufferedMutator.java index cf469a5..29a07e4 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/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/TypedBufferedMutator.java @@ -30,7 +30,7 @@ * * @param The class referring to the table to be written to. */ -class BufferedMutatorDelegator implements TypedBufferedMutator { +public class TypedBufferedMutator> { private final BufferedMutator bufferedMutator; @@ -38,7 +38,7 @@ * @param bufferedMutator the mutator to be wrapped for delegation. Shall not * be null. */ - public BufferedMutatorDelegator(BufferedMutator bufferedMutator) { + public TypedBufferedMutator(BufferedMutator bufferedMutator) { this.bufferedMutator = bufferedMutator; } 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/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTableRW.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 new file mode 100644 index 0000000..111ae71 --- /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/entity/EntityTableRW.java @@ -0,0 +1,136 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +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.BaseTableRW; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Create, read and write to the Entity Table. + */ +public class EntityTableRW extends BaseTableRW { + /** entity prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "entity"; + + /** config param name that specifies the entity table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** + * config param name that specifies the TTL for metrics column family in + * entity table. + */ + private static final String METRICS_TTL_CONF_NAME = PREFIX + + ".table.metrics.ttl"; + + /** + * config param name that specifies max-versions for metrics column family in + * entity table. + */ + private static final String METRICS_MAX_VERSIONS = + PREFIX + ".table.metrics.max-versions"; + + /** default value for entity table name. */ + public static final String DEFAULT_TABLE_NAME = "timelineservice.entity"; + + /** default TTL is 30 days for metrics timeseries. */ + private static final int DEFAULT_METRICS_TTL = 2592000; + + /** default max number of versions. */ + private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; + + private static final Logger LOG = + LoggerFactory.getLogger(EntityTableRW.class); + + public EntityTableRW() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW# + * createTable(org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor entityTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(EntityColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + entityTableDescp.addFamily(infoCF); + + HColumnDescriptor configCF = + new HColumnDescriptor(EntityColumnFamily.CONFIGS.getBytes()); + configCF.setBloomFilterType(BloomType.ROWCOL); + configCF.setBlockCacheEnabled(true); + entityTableDescp.addFamily(configCF); + + HColumnDescriptor metricsCF = + new HColumnDescriptor(EntityColumnFamily.METRICS.getBytes()); + entityTableDescp.addFamily(metricsCF); + metricsCF.setBlockCacheEnabled(true); + // always keep 1 version (the latest) + metricsCF.setMinVersions(1); + metricsCF.setMaxVersions( + hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); + metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, + DEFAULT_METRICS_TTL)); + entityTableDescp.setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + entityTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(entityTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } + + /** + * @param metricsTTL time to live parameter for the metricss in this table. + * @param hbaseConf configururation in which to set the metrics TTL config + * variable. + */ + public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { + hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); + } + +} 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/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTableRW.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 new file mode 100644 index 0000000..5b9fe13 --- /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/flow/FlowActivityTableRW.java @@ -0,0 +1,91 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +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.BaseTableRW; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Create, read and write to the FlowActivity Table. + */ +public class FlowActivityTableRW extends BaseTableRW { + /** flow activity table prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowactivity"; + + /** config param name that specifies the flowactivity table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for flowactivity table name. */ + public static final String DEFAULT_TABLE_NAME = + "timelineservice.flowactivity"; + + private static final Logger LOG = + LoggerFactory.getLogger(FlowActivityTableRW.class); + + /** default max number of versions. */ + public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; + + public FlowActivityTableRW() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW# + * createTable(org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor flowActivityTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(FlowActivityColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + flowActivityTableDescp.addFamily(infoCF); + infoCF.setMinVersions(1); + infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); + + // TODO: figure the split policy before running in production + admin.createTable(flowActivityTableDescp); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } +} \ No newline at end of file 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/flow/FlowRunTableRW.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 new file mode 100644 index 0000000..61c0734 --- /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/flow/FlowRunTableRW.java @@ -0,0 +1,102 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +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.BaseTableRW; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Coprocessor; + +/** + * Create, read and write to the FlowRun table. + */ +public class FlowRunTableRW extends BaseTableRW { + /** entity prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowrun"; + + /** config param name that specifies the flowrun table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for flowrun table name. */ + public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun"; + + private static final Logger LOG = + LoggerFactory.getLogger(FlowRunTableRW.class); + + /** default max number of versions. */ + public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; + + public FlowRunTableRW() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW# + * createTable(org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor flowRunTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(FlowRunColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + flowRunTableDescp.addFamily(infoCF); + infoCF.setMinVersions(1); + infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); + + // TODO: figure the split policy + String coprocessorJarPathStr = hbaseConf.get( + YarnConfiguration.FLOW_RUN_COPROCESSOR_JAR_HDFS_LOCATION, + YarnConfiguration.DEFAULT_HDFS_LOCATION_FLOW_RUN_COPROCESSOR_JAR); + + Path coprocessorJarPath = new Path(coprocessorJarPathStr); + LOG.info("CoprocessorJarPath=" + coprocessorJarPath.toString()); + 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 91% 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..0956f1e 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.ColumnRWHelper; 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 = ColumnRWHelper.readResult( + result, AppToFlowColumnPrefix.FLOW_NAME, clusterId); + Object flowRunId = ColumnRWHelper.readResult( + result, AppToFlowColumnPrefix.FLOW_RUN_ID, clusterId); + Object userId = ColumnRWHelper.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 0edd6a5..7440316 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 @@ -49,8 +49,9 @@ 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.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTableRW; +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; @@ -63,23 +64,23 @@ * 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, + ApplicationEntityReader(TimelineReaderContext ctxt, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { super(ctxt, entityFilters, toRetrieve); } - public ApplicationEntityReader(TimelineReaderContext ctxt, + ApplicationEntityReader(TimelineReaderContext ctxt, TimelineDataToRetrieve toRetrieve) { super(ctxt, toRetrieve); } /** - * Uses the {@link ApplicationTable}. + * Uses the {@link ApplicationTableRW}. */ - protected BaseTable getTable() { + protected BaseTableRW getTable() { return APPLICATION_TABLE; } @@ -430,12 +431,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 3f72334..ebe21a4 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 @@ -33,7 +33,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 org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,7 +51,7 @@ private static final Logger LOG = LoggerFactory.getLogger(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 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/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 a1cdb29..d0a0f3b 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 @@ -35,13 +35,14 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; -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.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 org.apache.hadoop.yarn.webapp.BadRequestException; import com.google.common.base.Preconditions; @@ -51,8 +52,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. @@ -60,21 +61,21 @@ private final KeyConverter longKeyConverter = new LongKeyConverter(); - public FlowActivityEntityReader(TimelineReaderContext ctxt, + FlowActivityEntityReader(TimelineReaderContext ctxt, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { super(ctxt, entityFilters, toRetrieve); } - public FlowActivityEntityReader(TimelineReaderContext ctxt, + FlowActivityEntityReader(TimelineReaderContext ctxt, TimelineDataToRetrieve toRetrieve) { super(ctxt, toRetrieve); } /** - * Uses the {@link FlowActivityTable}. + * Uses the {@link FlowActivityTableRW}. */ @Override - protected BaseTable getTable() { + protected BaseTableRW getTable() { return FLOW_ACTIVITY_TABLE; } @@ -164,8 +165,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 = ColumnRWHelper.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 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/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 af043b3..33a2cf6 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 @@ -43,7 +43,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; @@ -51,7 +52,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; @@ -61,23 +62,23 @@ * 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, + FlowRunEntityReader(TimelineReaderContext ctxt, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { super(ctxt, entityFilters, toRetrieve); } - public FlowRunEntityReader(TimelineReaderContext ctxt, + FlowRunEntityReader(TimelineReaderContext ctxt, TimelineDataToRetrieve toRetrieve) { super(ctxt, toRetrieve); } /** - * Uses the {@link FlowRunTable}. + * Uses the {@link FlowRunTableRW}. */ @Override - protected BaseTable getTable() { + protected BaseTableRW getTable() { return FLOW_RUN_TABLE; } @@ -261,19 +262,22 @@ protected TimelineEntity parseEntity(Result result) throws IOException { flowRun.setName(rowKey.getFlowName()); // 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 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/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 3a44445..02eca84 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 @@ -46,7 +46,9 @@ 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.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; @@ -57,7 +59,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.apache.hadoop.yarn.webapp.BadRequestException; import com.google.common.base.Preconditions; @@ -67,7 +69,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. @@ -75,20 +77,20 @@ private final KeyConverter stringKeyConverter = new StringKeyConverter(); - public GenericEntityReader(TimelineReaderContext ctxt, + GenericEntityReader(TimelineReaderContext ctxt, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { super(ctxt, entityFilters, toRetrieve); } - public GenericEntityReader(TimelineReaderContext ctxt, + GenericEntityReader(TimelineReaderContext ctxt, TimelineDataToRetrieve toRetrieve) { super(ctxt, toRetrieve); } /** - * Uses the {@link EntityTable}. + * Uses the {@link EntityTableRW}. */ - protected BaseTable getTable() { + protected BaseTableRW getTable() { return ENTITY_TABLE; } @@ -543,7 +545,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(); @@ -635,11 +638,12 @@ protected TimelineEntity parseEntity(Result result) throws IOException { * @param isConfig if true, means we are reading configs, otherwise info. * @throws IOException if any problem is encountered while reading result. */ - protected void readKeyValuePairs(TimelineEntity entity, Result result, + protected > void readKeyValuePairs( + TimelineEntity entity, Result result, ColumnPrefix prefix, boolean isConfig) throws IOException { // info and configuration are of type Map Map columns = - prefix.readResults(result, stringKeyConverter); + ColumnRWHelper.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/SubApplicationEntityReader.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/SubApplicationEntityReader.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/SubApplicationEntityReader.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/SubApplicationEntityReader.java index e780dcc..faed348 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/SubApplicationEntityReader.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/SubApplicationEntityReader.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.common.TimelineStorageUtils; @@ -51,14 +52,14 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix; import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTableRW; import org.apache.hadoop.yarn.webapp.BadRequestException; import com.google.common.base.Preconditions; class SubApplicationEntityReader extends GenericEntityReader { - private static final SubApplicationTable SUB_APPLICATION_TABLE = - new SubApplicationTable(); + private static final SubApplicationTableRW SUB_APPLICATION_TABLE = + new SubApplicationTableRW(); SubApplicationEntityReader(TimelineReaderContext ctxt, TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { @@ -71,9 +72,9 @@ } /** - * Uses the {@link SubApplicationTable}. + * Uses the {@link SubApplicationTableRW}. */ - protected BaseTable getTable() { + protected BaseTableRW getTable() { return SUB_APPLICATION_TABLE; } @@ -403,8 +404,8 @@ protected TimelineEntity parseEntity(Result result) throws IOException { TimelineEntityFilters filters = getFilters(); // fetch created time - Long createdTime = - (Long) SubApplicationColumn.CREATED_TIME.readResult(result); + Long createdTime = (Long) ColumnRWHelper.readResult(result, + SubApplicationColumn.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/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 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/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 07e8423..3168163 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 @@ -44,7 +44,9 @@ 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.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; @@ -72,7 +74,7 @@ /** * Main table the entity reader uses. */ - private BaseTable table; + private BaseTableRW table; /** * Used to convert strings key components to and from storage format. @@ -261,7 +263,7 @@ public TimelineEntity readEntity(Configuration hbaseConf, Connection conn) * * @return A reference to the table. */ - protected BaseTable getTable() { + protected BaseTableRW getTable() { return table; } @@ -314,8 +316,8 @@ protected abstract TimelineEntity parseEntity(Result result) protected void readMetrics(TimelineEntity entity, Result result, ColumnPrefix columnPrefix) throws IOException { NavigableMap> metricsResult = - columnPrefix.readResultsWithTimestamps( - result, stringKeyConverter); + ColumnRWHelper.readResultsWithTimestamps( + result, columnPrefix, stringKeyConverter); for (Map.Entry> metricResult: metricsResult.entrySet()) { TimelineMetric metric = new TimelineMetric(); @@ -340,7 +342,7 @@ public boolean isSingleEntityRead() { return singleEntityRead; } - protected void setTable(BaseTable baseTable) { + protected void setTable(BaseTableRW baseTable) { this.table = baseTable; } @@ -367,8 +369,9 @@ protected boolean hasField(EnumSet fieldsToRetrieve, * @param columns set of column qualifiers. * @return filter list. */ - protected FilterList createFiltersFromColumnQualifiers( - ColumnPrefix colPrefix, Set columns) { + protected > FilterList + createFiltersFromColumnQualifiers( + ColumnPrefix colPrefix, Set columns) { FilterList list = new FilterList(Operator.MUST_PASS_ONE); for (String column : columns) { // For columns which have compound column qualifiers (eg. events), we need @@ -381,8 +384,8 @@ protected boolean hasField(EnumSet fieldsToRetrieve, return list; } - protected byte[] createColQualifierPrefix(ColumnPrefix colPrefix, - String column) { + protected > byte[] createColQualifierPrefix( + ColumnPrefix colPrefix, String column) { if (colPrefix == ApplicationColumnPrefix.EVENT || colPrefix == EntityColumnPrefix.EVENT) { return new EventColumnName(column, null, null).getColumnQualifier(); @@ -402,11 +405,12 @@ protected boolean hasField(EnumSet fieldsToRetrieve, * isRelatedTo, otherwise its added to relatesTo. * @throws IOException if any problem is encountered while reading result. */ - protected void readRelationship(TimelineEntity entity, Result result, + 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 = ColumnRWHelper.readResults( + result, prefix, stringKeyConverter); for (Map.Entry column : columns.entrySet()) { for (String id : Separator.VALUES.splitEncoded(column.getValue() .toString())) { @@ -430,11 +434,12 @@ protected boolean hasField(EnumSet fieldsToRetrieve, * @param prefix column prefix. * @throws IOException if any problem is encountered while reading result. */ - protected static void readEvents(TimelineEntity entity, Result result, + 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 = ColumnRWHelper.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-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTableRW.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/subapplication/SubApplicationTableRW.java new file mode 100644 index 0000000..256b24b --- /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/subapplication/SubApplicationTableRW.java @@ -0,0 +1,137 @@ +/** + * 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.subapplication; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +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.BaseTableRW; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Create, read and write to the SubApplication table. + */ +public class SubApplicationTableRW extends BaseTableRW { + /** sub app prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "subapplication"; + + /** config param name that specifies the subapplication table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** + * config param name that specifies the TTL for metrics column family in + * subapplication table. + */ + private static final String METRICS_TTL_CONF_NAME = PREFIX + + ".table.metrics.ttl"; + + /** + * config param name that specifies max-versions for + * metrics column family in subapplication table. + */ + private static final String METRICS_MAX_VERSIONS = + PREFIX + ".table.metrics.max-versions"; + + /** default value for subapplication table name. */ + public static final String DEFAULT_TABLE_NAME = + "timelineservice.subapplication"; + + /** default TTL is 30 days for metrics timeseries. */ + private static final int DEFAULT_METRICS_TTL = 2592000; + + /** default max number of versions. */ + private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; + + private static final Logger LOG = LoggerFactory.getLogger( + SubApplicationTableRW.class); + + public SubApplicationTableRW() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW# + * createTable(org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor subAppTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(SubApplicationColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + subAppTableDescp.addFamily(infoCF); + + HColumnDescriptor configCF = + new HColumnDescriptor(SubApplicationColumnFamily.CONFIGS.getBytes()); + configCF.setBloomFilterType(BloomType.ROWCOL); + configCF.setBlockCacheEnabled(true); + subAppTableDescp.addFamily(configCF); + + HColumnDescriptor metricsCF = + new HColumnDescriptor(SubApplicationColumnFamily.METRICS.getBytes()); + subAppTableDescp.addFamily(metricsCF); + metricsCF.setBlockCacheEnabled(true); + // always keep 1 version (the latest) + metricsCF.setMinVersions(1); + metricsCF.setMaxVersions( + hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); + metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, + DEFAULT_METRICS_TTL)); + subAppTableDescp.setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + subAppTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(subAppTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } + + /** + * @param metricsTTL time to live parameter for the metricss in this table. + * @param hbaseConf configururation in which to set the metrics TTL config + * variable. + */ + public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { + hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); + } + +} 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/subapplication/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/subapplication/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/subapplication/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/subapplication/package-info.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/TestHBaseTimelineStorageUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.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/TestHBaseTimelineStorageUtils.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/pom.xml new file mode 100644 index 0000000..3c010ba --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/pom.xml @@ -0,0 +1,132 @@ + + + + + hadoop-yarn-server-timelineservice-hbase + org.apache.hadoop + 3.2.0-SNAPSHOT + + 4.0.0 + + hadoop-yarn-server-timelineservice-hbase-common + Apache Hadoop YARN TimelineService HBase Schema + 3.2.0-SNAPSHOT + + + + ${project.parent.parent.parent.basedir} + + + + + org.apache.hadoop + hadoop-annotations + provided + + + + org.apache.hadoop + hadoop-yarn-api + provided + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + + + + + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + provided + + + + + org.apache.hadoop + hadoop-common + test-jar + test + + + + org.apache.hbase + hbase-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.mortbay.jetty + jetty-util + + + + + + com.google.guava + guava + test + + + + junit + junit + test + + + + + + + maven-jar-plugin + + + + test-jar + + test-compile + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + package + + copy-dependencies + + + runtime + org.slf4j,org.apache.hadoop,com.github.stephenc.findbugs + ${project.build.directory}/lib + + + + + + + \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java new file mode 100644 index 0000000..c3d6a52 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java @@ -0,0 +1,101 @@ +/** + * 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 valueConverter; + + 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.valueConverter = 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 valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimestamp() { + return false; + } +} 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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java new file mode 100644 index 0000000..89412f4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java @@ -0,0 +1,150 @@ +/** + * 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 valueConverter; + + /** + * 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.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)); + } + } + + /** + * @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(); + } + + @Override + public byte[] getColumnPrefixInBytes() { + return columnPrefixBytes != null ? columnPrefixBytes.clone() : null; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimeStamp() { + return false; + } + + public ValueConverter getValueConverter() { + return valueConverter; + } +} 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-common/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-common/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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java new file mode 100644 index 0000000..16ab5fa --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The application table as column families info, config and metrics. Info + * stores information about a YARN application entity, config stores + * configuration data of a YARN application, metrics stores the metrics of a + * YARN application. This table is entirely analogous to the entity table but + * created for better performance. + * + * Example application table record: + * + *
+ * |-------------------------------------------------------------------------|
+ * |  Row       | Column Family                | Column Family| Column Family|
+ * |  key       | info                         | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | clusterId! | id:appId                     | metricId1:   | configKey1:  |
+ * | userName!  |                              | metricValue1 | configValue1 |
+ * | flowName!  | created_time:                | @timestamp1  |              |
+ * | flowRunId! | 1392993084018                |              | configKey2:  |
+ * | AppId      |                              | metriciD1:   | configValue2 |
+ * |            | i!infoKey:                   | metricValue2 |              |
+ * |            | infoValue                    | @timestamp2  |              |
+ * |            |                              |              |              |
+ * |            | r!relatesToKey:              | metricId2:   |              |
+ * |            | id3=id4=id5                  | metricValue1 |              |
+ * |            |                              | @timestamp2  |              |
+ * |            | s!isRelatedToKey:            |              |              |
+ * |            | id7=id9=id6                  |              |              |
+ * |            |                              |              |              |
+ * |            | e!eventId=timestamp=infoKey: |              |              |
+ * |            | eventInfoValue               |              |              |
+ * |            |                              |              |              |
+ * |            | flowVersion:                 |              |              |
+ * |            | versionValue                 |              |              |
+ * |-------------------------------------------------------------------------|
+ * 
+ */ +public final class ApplicationTable extends BaseTable { +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/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-common/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/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java new file mode 100644 index 0000000..0065f07 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java @@ -0,0 +1,95 @@ +/** + * 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.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.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 AppToFlowTable}. + */ +public enum AppToFlowColumn implements Column { + + /** + * The flow ID. + */ + FLOW_ID(AppToFlowColumnFamily.MAPPING, "flow_id"), + + /** + * 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 columnQualifier; + private final byte[] columnQualifierBytes; + private final ValueConverter valueConverter; + + AppToFlowColumn(ColumnFamily columnFamily, + String columnQualifier) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = + Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); + this.valueConverter = GenericConverter.getInstance(); + } + + /** + * @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 valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimestamp() { + return false; + } +} 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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java new file mode 100644 index 0000000..9540129 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java @@ -0,0 +1,105 @@ +/** + * 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 valueConverter; + + 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.valueConverter = GenericConverter.getInstance(); + } + + @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[] getColumnPrefixInBytes() { + return columnPrefixBytes != null ? columnPrefixBytes.clone() : null; + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimeStamp() { + return false; + } +} 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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java new file mode 100644 index 0000000..e184288 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The app_flow table as column families mapping. Mapping stores + * appId to flowName and flowRunId mapping information + * + * Example app_flow table record: + * + *
+ * |--------------------------------------|
+ * |  Row       | Column Family           |
+ * |  key       | mapping                 |
+ * |--------------------------------------|
+ * | appId      | flow_name!cluster1:     |
+ * |            | foo@daily_hive_report   |
+ * |            |                         |
+ * |            | flow_run_id!cluster1:   |
+ * |            | 1452828720457           |
+ * |            |                         |
+ * |            | user_id!cluster1:       |
+ * |            | admin                   |
+ * |            |                         |
+ * |            | flow_name!cluster2:     |
+ * |            | bar@ad_hoc_query        |
+ * |            |                         |
+ * |            | flow_run_id!cluster2:   |
+ * |            | 1452828498752           |
+ * |            |                         |
+ * |            | user_id!cluster2:       |
+ * |            | joe                     |
+ * |            |                         |
+ * |--------------------------------------|
+ * 
+ * + * It is possible (although unlikely) in a multi-cluster environment that there + * may be more than one applications for a given app id. Different clusters are + * recorded as different sets of columns. + */ +public final class AppToFlowTable extends BaseTable { +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.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/common/AppIdKeyConverter.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java index 51604f0..d934f74 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-common/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,9 +80,9 @@ 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 HBaseTimelineStorageUtils.convertApplicationIdToString( + return HBaseTimelineSchemaUtils.convertApplicationIdToString( ApplicationId.newInstance(clusterTs, seqId)); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java new file mode 100644 index 0000000..433b352 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java @@ -0,0 +1,27 @@ +/** + * 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; + + +/** + * The base type of tables. + * @param T table type + */ +public abstract class BaseTable { +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java new file mode 100644 index 0000000..2b50252 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java @@ -0,0 +1,56 @@ +/** + * 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[] getCombinedAttrsWithAggr(Attribute... attributes); + + /** + * Return true if the cell timestamp needs to be supplemented. + * @return true if the cell timestamp needs to be supplemented + */ + boolean supplementCellTimestamp(); +} \ 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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.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/common/ColumnFamily.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java index 452adcd..c5c8cb4 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/ColumnFamily.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java @@ -22,7 +22,7 @@ * * @param refers to the table for which this column family is used for. */ -public interface ColumnFamily { +public interface ColumnFamily> { /** * Keep a local copy if you need to avoid overhead of repeated cloning. diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java new file mode 100644 index 0000000..b173ef2 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java @@ -0,0 +1,101 @@ +/** + * 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.util.Bytes; + +/** + * This class is meant to be used only by explicit Columns, and not directly to + * write by clients. + */ +public final class ColumnHelper { + + 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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java new file mode 100644 index 0000000..cbcd936 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java @@ -0,0 +1,71 @@ +/** + * 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> { + + /** + * @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); + + /** + * Get the column prefix in bytes. + * @return column prefix in bytes + */ + byte[] getColumnPrefixInBytes(); + + /** + * 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[] getCombinedAttrsWithAggr(Attribute... attributes); + + /** + * Return true if the cell timestamp needs to be supplemented. + * @return true if the cell timestamp needs to be supplemented + */ + boolean supplementCellTimeStamp(); +} \ 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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineSchemaUtils.java new file mode 100644 index 0000000..e5f92cc --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineSchemaUtils.java @@ -0,0 +1,156 @@ +/** + * 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.api.records.ApplicationId; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +import java.text.NumberFormat; + +/** + * A bunch of utility functions used in HBase TimelineService common module. + */ +public final class HBaseTimelineSchemaUtils { + /** milliseconds in one day. */ + public static final long MILLIS_ONE_DAY = 86400000L; + + private static final ThreadLocal APP_ID_FORMAT = + new ThreadLocal() { + @Override + public NumberFormat initialValue() { + NumberFormat fmt = NumberFormat.getInstance(); + fmt.setGroupingUsed(false); + fmt.setMinimumIntegerDigits(4); + return fmt; + } + }; + + 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; + } + + /** + * 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); + } + + /** + * A utility method that converts ApplicationId to string without using + * FastNumberFormat in order to avoid the incompatibility issue caused + * by mixing hadoop-common 2.5.1 and hadoop-yarn-api 3.0 in this module. + * This is a work-around implementation as discussed in YARN-6905. + * + * @param appId application id + * @return the string representation of the given application id + * + */ + public static String convertApplicationIdToString(ApplicationId appId) { + StringBuilder sb = new StringBuilder(64); + sb.append(ApplicationId.appIdStrPrefix); + sb.append("_"); + sb.append(appId.getClusterTimestamp()); + sb.append('_'); + sb.append(APP_ID_FORMAT.get().format(appId.getId())); + return sb.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/common/KeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/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/KeyConverterToString.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverterToString.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/KeyConverterToString.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverterToString.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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java new file mode 100644 index 0000000..0df5b8a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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.common contains + * a set of utility classes used across backend storage reader and writer. + */ +@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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java new file mode 100644 index 0000000..81961d3 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java @@ -0,0 +1,105 @@ +/** + * 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 valueConverter; + + 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.valueConverter = 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 valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimestamp() { + return false; + } +} 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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java new file mode 100644 index 0000000..08234d3 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java @@ -0,0 +1,162 @@ +/** + * 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 valueConverter; + + /** + * 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.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)); + } + } + + /** + * @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[] getColumnPrefixInBytes() { + return columnPrefixBytes != null ? columnPrefixBytes.clone() : null; + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimeStamp() { + return false; + } +} 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-common/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-common/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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java new file mode 100644 index 0000000..dceeb99 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The entity table as column families info, config and metrics. Info stores + * information about a timeline entity object config stores configuration data + * of a timeline entity object metrics stores the metrics of a timeline entity + * object + * + * Example entity table record: + * + *
+ * |-------------------------------------------------------------------------|
+ * |  Row       | Column Family                | Column Family| Column Family|
+ * |  key       | info                         | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | userName!  | id:entityId                  | metricId1:   | configKey1:  |
+ * | clusterId! |                              | metricValue1 | configValue1 |
+ * | flowName!  | type:entityType              | @timestamp1  |              |
+ * | flowRunId! |                              |              | configKey2:  |
+ * | AppId!     | created_time:                | metricId1:   | configValue2 |
+ * | entityType!| 1392993084018                | metricValue2 |              |
+ * | idPrefix!  |                              | @timestamp2  |              |
+ * | entityId   | i!infoKey:                   |              |              |
+ * |            | infoValue                    | metricId1:   |              |
+ * |            |                              | metricValue1 |              |
+ * |            | r!relatesToKey:              | @timestamp2  |              |
+ * |            | id3=id4=id5                  |              |              |
+ * |            |                              |              |              |
+ * |            | s!isRelatedToKey             |              |              |
+ * |            | id7=id9=id6                  |              |              |
+ * |            |                              |              |              |
+ * |            | e!eventId=timestamp=infoKey: |              |              |
+ * |            | eventInfoValue               |              |              |
+ * |            |                              |              |              |
+ * |            | flowVersion:                 |              |              |
+ * |            | versionValue                 |              |              |
+ * |-------------------------------------------------------------------------|
+ * 
+ */ +public final class EntityTable extends BaseTable { +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java new file mode 100644 index 0000000..f468f0b --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java @@ -0,0 +1,133 @@ +/** + * 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; + private final ValueConverter valueConverter; + + /** + * 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) { + this.valueConverter = GenericConverter.getInstance(); + 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 byte[] getColumnPrefixInBytes() { + return columnPrefixBytes != null ? columnPrefixBytes.clone() : null; + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return HBaseTimelineSchemaUtils.combineAttributes(attributes, aggOp); + } + + @Override + public boolean supplementCellTimeStamp() { + return false; + } + + public AggregationOperation getAttribute() { + return aggOp; + } +} \ 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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java index b8a5dba..747f6ab 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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; -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.KeyConverterToString; @@ -63,7 +63,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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java new file mode 100644 index 0000000..e88a2fc --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The flow activity table has column family info + * Stores the daily activity record for flows + * Useful as a quick lookup of what flows were + * running on a given day + * + * Example flow activity table record: + * + *
+ * |-------------------------------------------|
+ * |  Row key   | Column Family                |
+ * |            | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | r!runid1:version1            |
+ * | inv Top of |                              |
+ * | Day!       | r!runid2:version7            |
+ * | userName!  |                              |
+ * | flowName   |                              |
+ * |-------------------------------------------|
+ * 
+ */ +public final class FlowActivityTable extends BaseTable { +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java new file mode 100644 index 0000000..2132d04 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java @@ -0,0 +1,112 @@ +/** + * 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 AggregationOperation aggOp; + private final ValueConverter valueConverter; + + 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.valueConverter = 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; + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return HBaseTimelineSchemaUtils.combineAttributes(attributes, aggOp); + } + + @Override + public boolean supplementCellTimestamp() { + return true; + } +} 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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java new file mode 100644 index 0000000..cc06bb4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java @@ -0,0 +1,129 @@ +/** + * 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 valueConverter; + + 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.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.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(); + } + + @Override + public byte[] getColumnPrefixInBytes() { + return columnPrefixBytes != null ? columnPrefixBytes.clone() : null; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return HBaseTimelineSchemaUtils.combineAttributes(attributes, aggOp); + } + + @Override + public boolean supplementCellTimeStamp() { + return true; + } + + public AggregationOperation getAttribute() { + return aggOp; + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } +} 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-common/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-common/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-common/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-common/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-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java new file mode 100644 index 0000000..643a102 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The flow run table has column family info + * Stores per flow run information + * aggregated across applications. + * + * Metrics are also stored in the info column family. + * + * Example flow run table record: + * + *
+ * flow_run table
+ * |-------------------------------------------|
+ * |  Row key   | Column Family                |
+ * |            | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | flow_version:version7        |
+ * | userName!  |                              |
+ * | flowName!  | running_apps:1               |
+ * | flowRunId  |                              |
+ * |            | min_start_time:1392995080000 |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | min_start_time:1392995081012 |
+ * |            | #0:appId2                    |
+ * |            |                              |
+ * |            | min_start_time:1392993083210 |
+ * |            | #0:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |            | max_end_time:1392993084018   |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            |                              |
+ * |            | m!mapInputRecords:127        |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | m!mapInputRecords:31         |
+ * |            | #2:appId2                    |
+ * |            |                              |
+ * |            | m!mapInputRecords:37         |
+ * |            | #1:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |            | m!mapOutputRecords:181       |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | m!mapOutputRecords:37        |
+ * |            | #1:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |-------------------------------------------|
+ * 
+ */ +public final class FlowRunTable extends BaseTable { +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/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-common/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-common/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/hadoop-yarn-server-timelineservice-hbase-common/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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java new file mode 100644 index 0000000..e78db2a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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; + +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-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.java new file mode 100644 index 0000000..a011a3f --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.java @@ -0,0 +1,99 @@ +/** + * 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.subapplication; + +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 SubApplicationTable}. + */ +public enum SubApplicationColumn implements Column { + + /** + * Identifier for the sub application. + */ + ID(SubApplicationColumnFamily.INFO, "id"), + + /** + * The type of sub application. + */ + TYPE(SubApplicationColumnFamily.INFO, "type"), + + /** + * When the sub application was created. + */ + CREATED_TIME(SubApplicationColumnFamily.INFO, "created_time", + new LongConverter()), + + /** + * The version of the flow that this sub application belongs to. + */ + FLOW_VERSION(SubApplicationColumnFamily.INFO, "flow_version"); + + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + private final ValueConverter valueConverter; + + SubApplicationColumn(ColumnFamily columnFamily, + String columnQualifier) { + this(columnFamily, columnQualifier, GenericConverter.getInstance()); + } + + SubApplicationColumn(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.valueConverter = converter; + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimestamp() { + return false; + } +} 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/subapplication/SubApplicationColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnFamily.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/subapplication/SubApplicationColumnFamily.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnFamily.java diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.java new file mode 100644 index 0000000..1106e37 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.java @@ -0,0 +1,163 @@ +/** + * 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.subapplication; + +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 sub app table. + */ +public enum SubApplicationColumnPrefix + implements ColumnPrefix { + + /** + * To store TimelineEntity getIsRelatedToEntities values. + */ + IS_RELATED_TO(SubApplicationColumnFamily.INFO, "s"), + + /** + * To store TimelineEntity getRelatesToEntities values. + */ + RELATES_TO(SubApplicationColumnFamily.INFO, "r"), + + /** + * To store TimelineEntity info values. + */ + INFO(SubApplicationColumnFamily.INFO, "i"), + + /** + * Lifecycle events for an entity. + */ + EVENT(SubApplicationColumnFamily.INFO, "e", true), + + /** + * Config column stores configuration with config key as the column name. + */ + CONFIG(SubApplicationColumnFamily.CONFIGS, null), + + /** + * Metrics are stored with the metric name as the column name. + */ + METRIC(SubApplicationColumnFamily.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 valueConverter; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + SubApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix) { + this(columnFamily, columnPrefix, false, GenericConverter.getInstance()); + } + + SubApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, boolean compondColQual) { + this(columnFamily, columnPrefix, compondColQual, + GenericConverter.getInstance()); + } + + SubApplicationColumnPrefix(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. + */ + SubApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, boolean compondColQual, ValueConverter converter) { + this.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)); + } + } + + /** + * @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[] getColumnPrefixInBytes() { + return columnPrefixBytes != null ? columnPrefixBytes.clone() : null; + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimeStamp() { + return false; + } +} 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/subapplication/SubApplicationRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKey.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/subapplication/SubApplicationRowKey.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKey.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/subapplication/SubApplicationRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKeyPrefix.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/subapplication/SubApplicationRowKeyPrefix.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKeyPrefix.java diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java new file mode 100644 index 0000000..de7dd4d --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java @@ -0,0 +1,64 @@ +/** + * 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.subapplication; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The sub application table has column families: + * info, config and metrics. + * Info stores information about a timeline entity object + * config stores configuration data of a timeline entity object + * metrics stores the metrics of a timeline entity object + * + * Example sub application table record: + * + *
+ * |-------------------------------------------------------------------------|
+ * |  Row          | Column Family             | Column Family| Column Family|
+ * |  key          | info                      | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | subAppUserId! | id:entityId               | metricId1:   | configKey1:  |
+ * | clusterId!    | type:entityType           | metricValue1 | configValue1 |
+ * | entityType!   |                           | @timestamp1  |              |
+ * | idPrefix!|    |                           |              | configKey2:  |
+ * | entityId!     | created_time:             | metricId1:   | configValue2 |
+ * | userId        | 1392993084018             | metricValue2 |              |
+ * |               |                           | @timestamp2  |              |
+ * |               | i!infoKey:                |              |              |
+ * |               | infoValue                 | metricId1:   |              |
+ * |               |                           | metricValue1 |              |
+ * |               |                           | @timestamp2  |              |
+ * |               | e!eventId=timestamp=      |              |              |
+ * |               | infoKey:                  |              |              |
+ * |               | eventInfoValue            |              |              |
+ * |               |                           |              |              |
+ * |               | r!relatesToKey:           |              |              |
+ * |               | id3=id4=id5               |              |              |
+ * |               |                           |              |              |
+ * |               | s!isRelatedToKey          |              |              |
+ * |               | id7=id9=id6               |              |              |
+ * |               |                           |              |              |
+ * |               | flowVersion:              |              |              |
+ * |               | versionValue              |              |              |
+ * |-------------------------------------------------------------------------|
+ * 
+ */ +public final class SubApplicationTable extends BaseTable { +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/package-info.java new file mode 100644 index 0000000..52cc399 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/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.subapplication + * contains classes related to implementation for subapplication table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.subapplication; + +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/TestCustomApplicationIdConversion.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java similarity index 95% 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/TestCustomApplicationIdConversion.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java index 73bc29e..0dc344f 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/TestCustomApplicationIdConversion.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java @@ -32,7 +32,7 @@ public void testConvertAplicationIdToString() { ApplicationId applicationId = ApplicationId.newInstance(0, 1); String applicationIdStr = - HBaseTimelineStorageUtils.convertApplicationIdToString(applicationId); + HBaseTimelineSchemaUtils.convertApplicationIdToString(applicationId); Assert.assertEquals(applicationId, ApplicationId.fromString(applicationIdStr)); } 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-common/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-common/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-common/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-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java index 4770238..d05cbad 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-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java @@ -195,7 +195,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/TestRowKeysAsString.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java similarity index 98% 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/TestRowKeysAsString.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java index 148cf56..c4d07c7 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/TestRowKeysAsString.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java @@ -94,7 +94,7 @@ public void testEntityRowKey() { @Test(timeout = 10000) public void testFlowActivityRowKey() { Long ts = 1459900830000L; - Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts); + Long dayTimestamp = HBaseTimelineSchemaUtils.getTopOfTheDayTimestamp(ts); String rowKeyAsString = new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME) .getRowKeyAsString(); FlowActivityRowKey rowKey = 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-common/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-common/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..218dc5a --- /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,129 @@ + + + + + hadoop-yarn-server-timelineservice-hbase + org.apache.hadoop + 3.2.0-SNAPSHOT + + + 4.0.0 + hadoop-yarn-server-timelineservice-hbase-server + Apache Hadoop YARN TimelineService HBase Server + 3.2.0-SNAPSHOT + + + + ${project.parent.parent.parent.basedir} + + + + + commons-logging + commons-logging + + + + com.google.guava + guava + + + + org.apache.hadoop + hadoop-common + provided + + + + org.apache.hadoop + hadoop-yarn-api + provided + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase-common + + + + 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 + provided + + + 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 + + + + + + \ No newline at end of file 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/common/HBaseTimelineServerUtils.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/common/HBaseTimelineServerUtils.java new file mode 100644 index 0000000..5c07670 --- /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/common/HBaseTimelineServerUtils.java @@ -0,0 +1,135 @@ +/** + * 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.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 java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * A utility class used by hbase-server module. + */ +public final class HBaseTimelineServerUtils { + private HBaseTimelineServerUtils() { + } + + /** + * 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; + } + + /** + * 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; + } +} 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/common/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/common/package-info.java new file mode 100644 index 0000000..0df5b8a --- /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/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.common contains + * a set of utility classes used across backend storage reader and writer. + */ +@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/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 99% 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 96a7cf3..41a371b 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 @@ -46,7 +46,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.HBaseTimelineServerUtils; import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -97,7 +97,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 = HBaseTimelineServerUtils.getTagFromAttribute(attribute); if (t != null) { tags.add(t); } 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 dbd0484..b228b5f 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 @@ -43,7 +43,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.HBaseTimelineServerUtils; 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; @@ -250,7 +250,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 HBaseTimelineServerUtils.getAggregationOperationFromTagsList(tags); } /** @@ -324,7 +324,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 = HBaseTimelineServerUtils .getAggregationCompactionDimension(tags); if (!alreadySeenAggDim.contains(aggDim)) { // if this agg dimension has already been seen, @@ -420,7 +420,7 @@ private Cell processSummation(SortedSet currentColumnCells, } byte[] sumBytes = converter.encodeValue(sum); Cell sumCell = - HBaseTimelineStorageUtils.createNewCell(mostRecentCell, sumBytes); + HBaseTimelineServerUtils.createNewCell(mostRecentCell, sumBytes); return sumCell; } @@ -462,7 +462,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 = HBaseTimelineServerUtils .getAggregationCompactionDimension(tags); if (appId == FLOW_APP_ID) { sum = converter.add(sum, currentValue); @@ -504,7 +504,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 = HBaseTimelineServerUtils.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 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/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 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/hadoop-yarn-server-timelineservice-hbase-server/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-server/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java new file mode 100644 index 0000000..e78db2a --- /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/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; + +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 6369864..7e5a803 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 @@ -18,197 +18,24 @@ + http://maven.apache.org/xsd/maven-4.0.0.xsd"> hadoop-yarn-server org.apache.hadoop 3.2.0-SNAPSHOT + 4.0.0 hadoop-yarn-server-timelineservice-hbase + 3.2.0-SNAPSHOT Apache Hadoop YARN TimelineService HBase Backend + pom - - - ${project.parent.parent.basedir} - - - - - commons-logging - commons-logging - - - - commons-lang - commons-lang - - - - commons-cli - commons-cli - - - - com.google.guava - guava - - - - org.apache.hadoop - hadoop-annotations - provided - - - - org.apache.hadoop - hadoop-common - provided - - - - - org.apache.hadoop - hadoop-common - test-jar - test - - - - org.apache.hadoop - hadoop-yarn-api - provided - - - - org.apache.hadoop - hadoop-yarn-common - provided - - - - org.apache.hadoop - hadoop-yarn-server-applicationhistoryservice - provided - - - - 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 - provided - - - 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 - - - + + hadoop-yarn-server-timelineservice-hbase-client + hadoop-yarn-server-timelineservice-hbase-common + hadoop-yarn-server-timelineservice-hbase-server + - - junit - junit - test - - - - - - maven-jar-plugin - - - - test-jar - - test-compile - - - - - org.apache.maven.plugins - maven-javadoc-plugin - - - - junit - junit - 4.11 - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - package - - copy-dependencies - - - runtime - org.slf4j,org.apache.hadoop,com.github.stephenc.findbugs - ${project.build.directory}/lib - - - - - - - + \ 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 f938185..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,593 +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.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.security.UserGroupInformation; -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.collector.TimelineCollectorContext; -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; -import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn; -import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * 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 Logger LOG = LoggerFactory - .getLogger(HBaseTimelineWriterImpl.class); - - private Connection conn; - private TypedBufferedMutator entityTable; - private TypedBufferedMutator appToFlowTable; - private TypedBufferedMutator applicationTable; - private TypedBufferedMutator flowActivityTable; - private TypedBufferedMutator flowRunTable; - private TypedBufferedMutator subApplicationTable; - - /** - * 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(); - - private enum Tables { - APPLICATION_TABLE, ENTITY_TABLE, SUBAPPLICATION_TABLE - }; - - public HBaseTimelineWriterImpl() { - super(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); - subApplicationTable = - new SubApplicationTable().getTableMutator(hbaseConf, conn); - - UserGroupInformation ugi = UserGroupInformation.isSecurityEnabled() ? - UserGroupInformation.getLoginUser() : - UserGroupInformation.getCurrentUser(); - LOG.info("Initialized HBaseTimelineWriterImpl UGI to " + ugi); - } - - /** - * Stores the entire information in TimelineEntities to the timeline store. - */ - @Override - public TimelineWriteResponse write(TimelineCollectorContext context, - TimelineEntities data, UserGroupInformation callerUgi) - throws IOException { - - TimelineWriteResponse putStatus = new TimelineWriteResponse(); - - String clusterId = context.getClusterId(); - String userId = context.getUserId(); - String flowName = context.getFlowName(); - String flowVersion = context.getFlowVersion(); - long flowRunId = context.getFlowRunId(); - String appId = context.getAppId(); - String subApplicationUser = callerUgi.getShortUserName(); - - // 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(); - store(rowKey, te, flowVersion, Tables.APPLICATION_TABLE); - } else { - EntityRowKey entityRowKey = - new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, - te.getType(), te.getIdPrefix(), te.getId()); - rowKey = entityRowKey.getRowKey(); - store(rowKey, te, flowVersion, Tables.ENTITY_TABLE); - } - - if (!isApplication && !userId.equals(subApplicationUser)) { - SubApplicationRowKey subApplicationRowKey = - new SubApplicationRowKey(subApplicationUser, clusterId, - te.getType(), te.getIdPrefix(), te.getId(), userId); - rowKey = subApplicationRowKey.getRowKey(); - store(rowKey, te, flowVersion, Tables.SUBAPPLICATION_TABLE); - } - - 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); - } - } - } - - /** - * Stores the Relations from the {@linkplain TimelineEntity} object. - */ - private void storeRelations(byte[] rowKey, - Map> connectedEntities, ColumnPrefix columnPrefix, - TypedBufferedMutator table) throws IOException { - if (connectedEntities != null) { - 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 store(byte[] rowKey, TimelineEntity te, - String flowVersion, - Tables table) throws IOException { - switch (table) { - case APPLICATION_TABLE: - 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); - storeInfo(rowKey, te.getInfo(), flowVersion, ApplicationColumnPrefix.INFO, - applicationTable); - storeMetrics(rowKey, te.getMetrics(), ApplicationColumnPrefix.METRIC, - applicationTable); - storeEvents(rowKey, te.getEvents(), ApplicationColumnPrefix.EVENT, - applicationTable); - storeConfig(rowKey, te.getConfigs(), ApplicationColumnPrefix.CONFIG, - applicationTable); - storeRelations(rowKey, te.getIsRelatedToEntities(), - ApplicationColumnPrefix.IS_RELATED_TO, applicationTable); - storeRelations(rowKey, te.getRelatesToEntities(), - ApplicationColumnPrefix.RELATES_TO, applicationTable); - break; - case ENTITY_TABLE: - 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); - storeInfo(rowKey, te.getInfo(), flowVersion, EntityColumnPrefix.INFO, - entityTable); - storeMetrics(rowKey, te.getMetrics(), EntityColumnPrefix.METRIC, - entityTable); - storeEvents(rowKey, te.getEvents(), EntityColumnPrefix.EVENT, - entityTable); - storeConfig(rowKey, te.getConfigs(), EntityColumnPrefix.CONFIG, - entityTable); - storeRelations(rowKey, te.getIsRelatedToEntities(), - EntityColumnPrefix.IS_RELATED_TO, entityTable); - storeRelations(rowKey, te.getRelatesToEntities(), - EntityColumnPrefix.RELATES_TO, entityTable); - break; - case SUBAPPLICATION_TABLE: - SubApplicationColumn.ID.store(rowKey, subApplicationTable, null, - te.getId()); - SubApplicationColumn.TYPE.store(rowKey, subApplicationTable, null, - te.getType()); - SubApplicationColumn.CREATED_TIME.store(rowKey, subApplicationTable, null, - te.getCreatedTime()); - SubApplicationColumn.FLOW_VERSION.store(rowKey, subApplicationTable, null, - flowVersion); - storeInfo(rowKey, te.getInfo(), flowVersion, - SubApplicationColumnPrefix.INFO, subApplicationTable); - storeMetrics(rowKey, te.getMetrics(), SubApplicationColumnPrefix.METRIC, - subApplicationTable); - storeEvents(rowKey, te.getEvents(), SubApplicationColumnPrefix.EVENT, - subApplicationTable); - storeConfig(rowKey, te.getConfigs(), SubApplicationColumnPrefix.CONFIG, - subApplicationTable); - storeRelations(rowKey, te.getIsRelatedToEntities(), - SubApplicationColumnPrefix.IS_RELATED_TO, subApplicationTable); - storeRelations(rowKey, te.getRelatesToEntities(), - SubApplicationColumnPrefix.RELATES_TO, subApplicationTable); - break; - default: - LOG.info("Invalid table name provided."); - break; - } - } - - /** - * stores the info information from {@linkplain TimelineEntity}. - */ - private void storeInfo(byte[] rowKey, Map info, - String flowVersion, ColumnPrefix columnPrefix, - TypedBufferedMutator table) throws IOException { - if (info != null) { - for (Map.Entry entry : info.entrySet()) { - columnPrefix.store(rowKey, table, - stringKeyConverter.encode(entry.getKey()), null, entry.getValue()); - } - } - } - - /** - * stores the config information from {@linkplain TimelineEntity}. - */ - private void storeConfig(byte[] rowKey, Map config, - ColumnPrefix columnPrefix, TypedBufferedMutator table) - throws IOException { - if (config != null) { - for (Map.Entry entry : config.entrySet()) { - byte[] configKey = stringKeyConverter.encode(entry.getKey()); - columnPrefix.store(rowKey, table, configKey, null, entry.getValue()); - } - } - } - - /** - * stores the {@linkplain TimelineMetric} information from the - * {@linkplain TimelineEvent} object. - */ - private void storeMetrics(byte[] rowKey, Set metrics, - ColumnPrefix columnPrefix, TypedBufferedMutator table) - 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(); - columnPrefix.store(rowKey, table, metricColumnQualifier, timestamp, - timeseriesEntry.getValue()); - } - } - } - } - - /** - * Stores the events from the {@linkplain TimelineEvent} object. - */ - private void storeEvents(byte[] rowKey, Set events, - ColumnPrefix columnPrefix, TypedBufferedMutator table) - 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(); - columnPrefix.store(rowKey, table, columnQualifierBytes, null, - Separator.EMPTY_BYTES); - } else { - for (Map.Entry info : eventInfo.entrySet()) { - // eventId=infoKey - byte[] columnQualifierBytes = - new EventColumnName(eventId, eventTimestamp, info.getKey()) - .getColumnQualifier(); - columnPrefix.store(rowKey, table, 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(); - subApplicationTable.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 (subApplicationTable != null) { - subApplicationTable.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 00eaa7e..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,108 +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(); - } - -} 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 8297dc5..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,236 +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); - } - -} 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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java deleted file mode 100644 index 4da720e..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/ApplicationTable.java +++ /dev/null @@ -1,170 +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.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -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.TimelineHBaseSchemaConstants; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The application table as column families info, config and metrics. Info - * stores information about a YARN application entity, config stores - * configuration data of a YARN application, metrics stores the metrics of a - * YARN application. This table is entirely analogous to the entity table but - * created for better performance. - * - * Example application table record: - * - *
- * |-------------------------------------------------------------------------|
- * |  Row       | Column Family                | Column Family| Column Family|
- * |  key       | info                         | metrics      | config       |
- * |-------------------------------------------------------------------------|
- * | clusterId! | id:appId                     | metricId1:   | configKey1:  |
- * | userName!  |                              | metricValue1 | configValue1 |
- * | flowName!  | created_time:                | @timestamp1  |              |
- * | flowRunId! | 1392993084018                |              | configKey2:  |
- * | AppId      |                              | metriciD1:   | configValue2 |
- * |            | i!infoKey:                   | metricValue2 |              |
- * |            | infoValue                    | @timestamp2  |              |
- * |            |                              |              |              |
- * |            | r!relatesToKey:              | metricId2:   |              |
- * |            | id3=id4=id5                  | metricValue1 |              |
- * |            |                              | @timestamp2  |              |
- * |            | s!isRelatedToKey:            |              |              |
- * |            | id7=id9=id6                  |              |              |
- * |            |                              |              |              |
- * |            | e!eventId=timestamp=infoKey: |              |              |
- * |            | eventInfoValue               |              |              |
- * |            |                              |              |              |
- * |            | flowVersion:                 |              |              |
- * |            | versionValue                 |              |              |
- * |-------------------------------------------------------------------------|
- * 
- */ -public class ApplicationTable extends BaseTable { - /** application prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + "application"; - - /** config param name that specifies the application table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** - * config param name that specifies the TTL for metrics column family in - * application table. - */ - private static final String METRICS_TTL_CONF_NAME = PREFIX - + ".table.metrics.ttl"; - - /** - * config param name that specifies max-versions for metrics column family in - * entity table. - */ - private static final String METRICS_MAX_VERSIONS = - PREFIX + ".table.metrics.max-versions"; - - /** default value for application table name. */ - private static final String DEFAULT_TABLE_NAME = - "timelineservice.application"; - - /** default TTL is 30 days for metrics timeseries. */ - private static final int DEFAULT_METRICS_TTL = 2592000; - - /** default max number of versions. */ - private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; - - private static final Logger LOG = - LoggerFactory.getLogger(ApplicationTable.class); - - public ApplicationTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor applicationTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(ApplicationColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - applicationTableDescp.addFamily(infoCF); - - HColumnDescriptor configCF = - new HColumnDescriptor(ApplicationColumnFamily.CONFIGS.getBytes()); - configCF.setBloomFilterType(BloomType.ROWCOL); - configCF.setBlockCacheEnabled(true); - applicationTableDescp.addFamily(configCF); - - HColumnDescriptor metricsCF = - new HColumnDescriptor(ApplicationColumnFamily.METRICS.getBytes()); - applicationTableDescp.addFamily(metricsCF); - metricsCF.setBlockCacheEnabled(true); - // always keep 1 version (the latest) - metricsCF.setMinVersions(1); - metricsCF.setMaxVersions( - hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); - metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, - DEFAULT_METRICS_TTL)); - applicationTableDescp.setRegionSplitPolicyClassName( - "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); - applicationTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", - TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); - admin.createTable(applicationTableDescp, - TimelineHBaseSchemaConstants.getUsernameSplits()); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } - - /** - * @param metricsTTL time to live parameter for the metrics in this table. - * @param hbaseConf configuration in which to set the metrics TTL config - * variable. - */ - public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { - hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); - } - -} 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/AppToFlowColumn.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/AppToFlowColumn.java deleted file mode 100644 index 67497fc..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/AppToFlowColumn.java +++ /dev/null @@ -1,101 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; - - -import org.apache.hadoop.hbase.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.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; - -import java.io.IOException; - -/** - * Identifies fully qualified columns for the {@link AppToFlowTable}. - */ -public enum AppToFlowColumn implements Column { - - /** - * The flow ID. - */ - FLOW_ID(AppToFlowColumnFamily.MAPPING, "flow_id"), - - /** - * 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 columnQualifier; - private final byte[] columnQualifierBytes; - - AppToFlowColumn(ColumnFamily columnFamily, - String columnQualifier) { - 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); - } - - /** - * @return the column name value - */ - private String getColumnQualifier() { - return columnQualifier; - } - - @Override - public byte[] getColumnQualifierBytes() { - return columnQualifierBytes.clone(); - } - - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, Long timestamp, - Object inputValue, Attribute... attributes) throws IOException { - column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, - inputValue, attributes); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - public Object readResult(Result result) throws IOException { - return column.readResult(result, columnQualifierBytes); - } - -} 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 752a380..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; - - 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/apptoflow/AppToFlowTable.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/AppToFlowTable.java deleted file mode 100644 index 04da5c7..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/AppToFlowTable.java +++ /dev/null @@ -1,125 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; - - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -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.TimelineHBaseSchemaConstants; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; - -/** - * The app_flow table as column families mapping. Mapping stores - * appId to flowName and flowRunId mapping information - * - * Example app_flow table record: - * - *
- * |--------------------------------------|
- * |  Row       | Column Family           |
- * |  key       | mapping                 |
- * |--------------------------------------|
- * | appId      | flow_name!cluster1:     |
- * |            | foo@daily_hive_report   |
- * |            |                         |
- * |            | flow_run_id!cluster1:   |
- * |            | 1452828720457           |
- * |            |                         |
- * |            | user_id!cluster1:       |
- * |            | admin                   |
- * |            |                         |
- * |            | flow_name!cluster2:     |
- * |            | bar@ad_hoc_query        |
- * |            |                         |
- * |            | flow_run_id!cluster2:   |
- * |            | 1452828498752           |
- * |            |                         |
- * |            | user_id!cluster2:       |
- * |            | joe                     |
- * |            |                         |
- * |--------------------------------------|
- * 
- * - * It is possible (although unlikely) in a multi-cluster environment that there - * 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 { - /** app_flow prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + "app-flow"; - - /** config param name that specifies the app_flow table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** default value for app_flow table name. */ - private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow"; - - private static final Logger LOG = - LoggerFactory.getLogger(AppToFlowTable.class); - - public AppToFlowTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor appToFlowTableDescp = new HTableDescriptor(table); - HColumnDescriptor mappCF = - new HColumnDescriptor(AppToFlowColumnFamily.MAPPING.getBytes()); - mappCF.setBloomFilterType(BloomType.ROWCOL); - appToFlowTableDescp.addFamily(mappCF); - - appToFlowTableDescp - .setRegionSplitPolicyClassName( - "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); - appToFlowTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", - TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); - admin.createTable(appToFlowTableDescp, - TimelineHBaseSchemaConstants.getUsernameSplits()); - 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/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 9f95d44..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,414 +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.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 org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * 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 Logger LOG = - LoggerFactory.getLogger(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; - - private final boolean supplementTs; - - public ColumnHelper(ColumnFamily columnFamily) { - this(columnFamily, GenericConverter.getInstance()); - } - - public ColumnHelper(ColumnFamily columnFamily, ValueConverter converter) { - this(columnFamily, converter, false); - } - - /** - * @param columnFamily column family implementation. - * @param converter converter use to encode/decode values stored in the column - * or column prefix. - * @param needSupplementTs flag to indicate if cell timestamp needs to be - * modified for this column by calling - * {@link TimestampGenerator#getSupplementedTimestamp(long, String)}. This - * would be required for columns(such as metrics in flow run table) where - * potential collisions can occur due to same timestamp. - */ - public ColumnHelper(ColumnFamily columnFamily, ValueConverter converter, - boolean needSupplementTs) { - this.columnFamily = columnFamily; - columnFamilyBytes = columnFamily.getBytes(); - if (converter == null) { - this.converter = GenericConverter.getInstance(); - } else { - this.converter = converter; - } - this.supplementTs = needSupplementTs; - } - - /** - * 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. - * Will supplement the timestamp if required. Typically done for flow run - * table.If we supplement the timestamp, we 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(); - } - if (!this.supplementTs) { - return timestamp; - } else { - 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()); - Long ts = supplementTs ? TimestampGenerator. - getTruncatedTimestamp(cell.getKey()) : cell.getKey(); - cellResults.put(ts, 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 c115b18..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,354 +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.net.MalformedURLException; -import java.net.URL; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -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.client.Query; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.api.records.ApplicationId; -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 org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.text.NumberFormat; - -/** - * 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 static final Logger LOG = - LoggerFactory.getLogger(HBaseTimelineStorageUtils.class); - - 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; - } - - private static final ThreadLocal APP_ID_FORMAT = - new ThreadLocal() { - @Override - public NumberFormat initialValue() { - NumberFormat fmt = NumberFormat.getInstance(); - fmt.setGroupingUsed(false); - fmt.setMinimumIntegerDigits(4); - return fmt; - } - }; - - /** - * A utility method that converts ApplicationId to string without using - * FastNumberFormat in order to avoid the incompatibility issue caused - * by mixing hadoop-common 2.5.1 and hadoop-yarn-api 3.0 in this module. - * This is a work-around implementation as discussed in YARN-6905. - * - * @param appId application id - * @return the string representation of the given application id - * - */ - public static String convertApplicationIdToString(ApplicationId appId) { - StringBuilder sb = new StringBuilder(64); - sb.append(ApplicationId.appIdStrPrefix); - sb.append("_"); - sb.append(appId.getClusterTimestamp()); - sb.append('_'); - sb.append(APP_ID_FORMAT.get().format(appId.getId())); - return sb.toString(); - } - - /** - * @param conf YARN configuration. Used to see if there is an explicit config - * pointing to the HBase config file to read. It should not be null - * or a NullPointerException will be thrown. - * @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 { - if (conf == null) { - throw new NullPointerException(); - } - - Configuration hbaseConf; - String timelineServiceHBaseConfFileURL = - conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE); - if (timelineServiceHBaseConfFileURL != null - && timelineServiceHBaseConfFileURL.length() > 0) { - LOG.info("Using hbase configuration at " + - timelineServiceHBaseConfFileURL); - // 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); - } - - public static void setMetricsTimeRange(Query query, byte[] metricsCf, - long tsBegin, long tsEnd) { - if (tsBegin != 0 || tsEnd != Long.MAX_VALUE) { - query.setColumnFamilyTimeRange(metricsCf, - tsBegin, ((tsEnd == Long.MAX_VALUE) ? Long.MAX_VALUE : (tsEnd + 1))); - } - } -} 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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java deleted file mode 100644 index 64a11f8..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/TypedBufferedMutator.java +++ /dev/null @@ -1,28 +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.hbase.client.BufferedMutator; - -/** - * Just a typed wrapper around {@link BufferedMutator} used to ensure that - * columns can write only to the table mutator for the right table. - */ -public interface TypedBufferedMutator extends BufferedMutator { - // This class is intentionally left (almost) blank -} 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 b228d84..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,112 +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); - } - - @Override - public byte[] getColumnQualifierBytes() { - return columnQualifierBytes.clone(); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - -} 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 d385108..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,249 +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); - } - -} 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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java deleted file mode 100644 index 988bba2..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/EntityTable.java +++ /dev/null @@ -1,170 +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.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -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.TimelineHBaseSchemaConstants; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The entity table as column families info, config and metrics. Info stores - * information about a timeline entity object config stores configuration data - * of a timeline entity object metrics stores the metrics of a timeline entity - * object - * - * Example entity table record: - * - *
- * |-------------------------------------------------------------------------|
- * |  Row       | Column Family                | Column Family| Column Family|
- * |  key       | info                         | metrics      | config       |
- * |-------------------------------------------------------------------------|
- * | userName!  | id:entityId                  | metricId1:   | configKey1:  |
- * | clusterId! |                              | metricValue1 | configValue1 |
- * | flowName!  | type:entityType              | @timestamp1  |              |
- * | flowRunId! |                              |              | configKey2:  |
- * | AppId!     | created_time:                | metricId1:   | configValue2 |
- * | entityType!| 1392993084018                | metricValue2 |              |
- * | idPrefix!  |                              | @timestamp2  |              |
- * | entityId   | i!infoKey:                   |              |              |
- * |            | infoValue                    | metricId1:   |              |
- * |            |                              | metricValue1 |              |
- * |            | r!relatesToKey:              | @timestamp2  |              |
- * |            | id3=id4=id5                  |              |              |
- * |            |                              |              |              |
- * |            | s!isRelatedToKey             |              |              |
- * |            | id7=id9=id6                  |              |              |
- * |            |                              |              |              |
- * |            | e!eventId=timestamp=infoKey: |              |              |
- * |            | eventInfoValue               |              |              |
- * |            |                              |              |              |
- * |            | flowVersion:                 |              |              |
- * |            | versionValue                 |              |              |
- * |-------------------------------------------------------------------------|
- * 
- */ -public class EntityTable extends BaseTable { - /** entity prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + "entity"; - - /** config param name that specifies the entity table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** - * config param name that specifies the TTL for metrics column family in - * entity table. - */ - private static final String METRICS_TTL_CONF_NAME = PREFIX - + ".table.metrics.ttl"; - - /** - * config param name that specifies max-versions for metrics column family in - * entity table. - */ - private static final String METRICS_MAX_VERSIONS = - PREFIX + ".table.metrics.max-versions"; - - /** default value for entity table name. */ - public static final String DEFAULT_TABLE_NAME = "timelineservice.entity"; - - /** default TTL is 30 days for metrics timeseries. */ - private static final int DEFAULT_METRICS_TTL = 2592000; - - /** default max number of versions. */ - private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; - - private static final Logger LOG = - LoggerFactory.getLogger(EntityTable.class); - - public EntityTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor entityTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(EntityColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - entityTableDescp.addFamily(infoCF); - - HColumnDescriptor configCF = - new HColumnDescriptor(EntityColumnFamily.CONFIGS.getBytes()); - configCF.setBloomFilterType(BloomType.ROWCOL); - configCF.setBlockCacheEnabled(true); - entityTableDescp.addFamily(configCF); - - HColumnDescriptor metricsCF = - new HColumnDescriptor(EntityColumnFamily.METRICS.getBytes()); - entityTableDescp.addFamily(metricsCF); - metricsCF.setBlockCacheEnabled(true); - // always keep 1 version (the latest) - metricsCF.setMinVersions(1); - metricsCF.setMaxVersions( - hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); - metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, - DEFAULT_METRICS_TTL)); - entityTableDescp.setRegionSplitPolicyClassName( - "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); - entityTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", - TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); - admin.createTable(entityTableDescp, - TimelineHBaseSchemaConstants.getUsernameSplits()); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } - - /** - * @param metricsTTL time to live parameter for the metricss in this table. - * @param hbaseConf configururation in which to set the metrics TTL config - * variable. - */ - public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { - hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); - } - -} 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 706b002..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,221 +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); - } - - /* - * (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, timestamp, 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/FlowActivityTable.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/FlowActivityTable.java deleted file mode 100644 index e646eb2..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/FlowActivityTable.java +++ /dev/null @@ -1,109 +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.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -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.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The flow activity table has column family info - * Stores the daily activity record for flows - * Useful as a quick lookup of what flows were - * running on a given day - * - * Example flow activity table record: - * - *
- * |-------------------------------------------|
- * |  Row key   | Column Family                |
- * |            | info                         |
- * |-------------------------------------------|
- * | clusterId! | r!runid1:version1            |
- * | inv Top of |                              |
- * | Day!       | r!runid2:version7            |
- * | userName!  |                              |
- * | flowName   |                              |
- * |-------------------------------------------|
- * 
- */ -public class FlowActivityTable extends BaseTable { - /** flow activity table prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowactivity"; - - /** config param name that specifies the flowactivity table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** default value for flowactivity table name. */ - public static final String DEFAULT_TABLE_NAME = - "timelineservice.flowactivity"; - - private static final Logger LOG = - LoggerFactory.getLogger(FlowActivityTable.class); - - /** default max number of versions. */ - public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; - - public FlowActivityTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor flowActivityTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(FlowActivityColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - flowActivityTableDescp.addFamily(infoCF); - infoCF.setMinVersions(1); - infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); - - // TODO: figure the split policy before running in production - admin.createTable(flowActivityTableDescp); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } -} \ 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 3797faf..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,131 +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, true); - } - - /** - * @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); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - -} 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 f521cd7..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,217 +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, true); - 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); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - -} 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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java deleted file mode 100644 index a1d32ee..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/FlowRunTable.java +++ /dev/null @@ -1,151 +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.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -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.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Coprocessor; - -/** - * The flow run table has column family info - * Stores per flow run information - * aggregated across applications. - * - * Metrics are also stored in the info column family. - * - * Example flow run table record: - * - *
- * flow_run table
- * |-------------------------------------------|
- * |  Row key   | Column Family                |
- * |            | info                         |
- * |-------------------------------------------|
- * | clusterId! | flow_version:version7        |
- * | userName!  |                              |
- * | flowName!  | running_apps:1               |
- * | flowRunId  |                              |
- * |            | min_start_time:1392995080000 |
- * |            | #0:""                        |
- * |            |                              |
- * |            | min_start_time:1392995081012 |
- * |            | #0:appId2                    |
- * |            |                              |
- * |            | min_start_time:1392993083210 |
- * |            | #0:appId3                    |
- * |            |                              |
- * |            |                              |
- * |            | max_end_time:1392993084018   |
- * |            | #0:""                        |
- * |            |                              |
- * |            |                              |
- * |            | m!mapInputRecords:127        |
- * |            | #0:""                        |
- * |            |                              |
- * |            | m!mapInputRecords:31         |
- * |            | #2:appId2                    |
- * |            |                              |
- * |            | m!mapInputRecords:37         |
- * |            | #1:appId3                    |
- * |            |                              |
- * |            |                              |
- * |            | m!mapOutputRecords:181       |
- * |            | #0:""                        |
- * |            |                              |
- * |            | m!mapOutputRecords:37        |
- * |            | #1:appId3                    |
- * |            |                              |
- * |            |                              |
- * |-------------------------------------------|
- * 
- */ -public class FlowRunTable extends BaseTable { - /** entity prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowrun"; - - /** config param name that specifies the flowrun table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** default value for flowrun table name. */ - public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun"; - - private static final Logger LOG = - LoggerFactory.getLogger(FlowRunTable.class); - - /** default max number of versions. */ - public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; - - public FlowRunTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor flowRunTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(FlowRunColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - flowRunTableDescp.addFamily(infoCF); - infoCF.setMinVersions(1); - infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); - - // TODO: figure the split policy - String coprocessorJarPathStr = hbaseConf.get( - YarnConfiguration.FLOW_RUN_COPROCESSOR_JAR_HDFS_LOCATION, - YarnConfiguration.DEFAULT_HDFS_LOCATION_FLOW_RUN_COPROCESSOR_JAR); - - Path coprocessorJarPath = new Path(coprocessorJarPathStr); - LOG.info("CoprocessorJarPath=" + coprocessorJarPath.toString()); - flowRunTableDescp.addCoprocessor( - FlowRunCoprocessor.class.getCanonicalName(), 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/subapplication/SubApplicationColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.java deleted file mode 100644 index 46b0cc9..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.java +++ /dev/null @@ -1,108 +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.subapplication; - -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 SubApplicationTable}. - */ -public enum SubApplicationColumn implements Column { - - /** - * Identifier for the sub application. - */ - ID(SubApplicationColumnFamily.INFO, "id"), - - /** - * The type of sub application. - */ - TYPE(SubApplicationColumnFamily.INFO, "type"), - - /** - * When the sub application was created. - */ - CREATED_TIME(SubApplicationColumnFamily.INFO, "created_time", - new LongConverter()), - - /** - * The version of the flow that this sub application belongs to. - */ - FLOW_VERSION(SubApplicationColumnFamily.INFO, "flow_version"); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - private final String columnQualifier; - private final byte[] columnQualifierBytes; - - SubApplicationColumn(ColumnFamily columnFamily, - String columnQualifier) { - this(columnFamily, columnQualifier, GenericConverter.getInstance()); - } - - SubApplicationColumn(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); - } - - - 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(); - } - -} 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/subapplication/SubApplicationColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.java deleted file mode 100644 index 06ecced..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.java +++ /dev/null @@ -1,250 +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.subapplication; - -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 sub app table. - */ -public enum SubApplicationColumnPrefix - implements ColumnPrefix { - - /** - * To store TimelineEntity getIsRelatedToEntities values. - */ - IS_RELATED_TO(SubApplicationColumnFamily.INFO, "s"), - - /** - * To store TimelineEntity getRelatesToEntities values. - */ - RELATES_TO(SubApplicationColumnFamily.INFO, "r"), - - /** - * To store TimelineEntity info values. - */ - INFO(SubApplicationColumnFamily.INFO, "i"), - - /** - * Lifecycle events for an entity. - */ - EVENT(SubApplicationColumnFamily.INFO, "e", true), - - /** - * Config column stores configuration with config key as the column name. - */ - CONFIG(SubApplicationColumnFamily.CONFIGS, null), - - /** - * Metrics are stored with the metric name as the column name. - */ - METRIC(SubApplicationColumnFamily.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. - */ - SubApplicationColumnPrefix(ColumnFamily columnFamily, - String columnPrefix) { - this(columnFamily, columnPrefix, false, GenericConverter.getInstance()); - } - - SubApplicationColumnPrefix(ColumnFamily columnFamily, - String columnPrefix, boolean compondColQual) { - this(columnFamily, columnPrefix, compondColQual, - GenericConverter.getInstance()); - } - - SubApplicationColumnPrefix(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. - */ - SubApplicationColumnPrefix(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); - } - -} 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/subapplication/SubApplicationTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java deleted file mode 100644 index 785a243..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java +++ /dev/null @@ -1,174 +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.subapplication; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -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.TimelineHBaseSchemaConstants; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The sub application table has column families: - * info, config and metrics. - * Info stores information about a timeline entity object - * config stores configuration data of a timeline entity object - * metrics stores the metrics of a timeline entity object - * - * Example sub application table record: - * - *
- * |-------------------------------------------------------------------------|
- * |  Row          | Column Family             | Column Family| Column Family|
- * |  key          | info                      | metrics      | config       |
- * |-------------------------------------------------------------------------|
- * | subAppUserId! | id:entityId               | metricId1:   | configKey1:  |
- * | clusterId!    | type:entityType           | metricValue1 | configValue1 |
- * | entityType!   |                           | @timestamp1  |              |
- * | idPrefix!|    |                           |              | configKey2:  |
- * | entityId!     | created_time:             | metricId1:   | configValue2 |
- * | userId        | 1392993084018             | metricValue2 |              |
- * |               |                           | @timestamp2  |              |
- * |               | i!infoKey:                |              |              |
- * |               | infoValue                 | metricId1:   |              |
- * |               |                           | metricValue1 |              |
- * |               |                           | @timestamp2  |              |
- * |               | e!eventId=timestamp=      |              |              |
- * |               | infoKey:                  |              |              |
- * |               | eventInfoValue            |              |              |
- * |               |                           |              |              |
- * |               | r!relatesToKey:           |              |              |
- * |               | id3=id4=id5               |              |              |
- * |               |                           |              |              |
- * |               | s!isRelatedToKey          |              |              |
- * |               | id7=id9=id6               |              |              |
- * |               |                           |              |              |
- * |               | flowVersion:              |              |              |
- * |               | versionValue              |              |              |
- * |-------------------------------------------------------------------------|
- * 
- */ -public class SubApplicationTable extends BaseTable { - /** sub app prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + "subapplication"; - - /** config param name that specifies the subapplication table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** - * config param name that specifies the TTL for metrics column family in - * subapplication table. - */ - private static final String METRICS_TTL_CONF_NAME = PREFIX - + ".table.metrics.ttl"; - - /** - * config param name that specifies max-versions for - * metrics column family in subapplication table. - */ - private static final String METRICS_MAX_VERSIONS = - PREFIX + ".table.metrics.max-versions"; - - /** default value for subapplication table name. */ - public static final String DEFAULT_TABLE_NAME = - "timelineservice.subapplication"; - - /** default TTL is 30 days for metrics timeseries. */ - private static final int DEFAULT_METRICS_TTL = 2592000; - - /** default max number of versions. */ - private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; - - private static final Logger LOG = LoggerFactory.getLogger( - SubApplicationTable.class); - - public SubApplicationTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor subAppTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(SubApplicationColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - subAppTableDescp.addFamily(infoCF); - - HColumnDescriptor configCF = - new HColumnDescriptor(SubApplicationColumnFamily.CONFIGS.getBytes()); - configCF.setBloomFilterType(BloomType.ROWCOL); - configCF.setBlockCacheEnabled(true); - subAppTableDescp.addFamily(configCF); - - HColumnDescriptor metricsCF = - new HColumnDescriptor(SubApplicationColumnFamily.METRICS.getBytes()); - subAppTableDescp.addFamily(metricsCF); - metricsCF.setBlockCacheEnabled(true); - // always keep 1 version (the latest) - metricsCF.setMinVersions(1); - metricsCF.setMaxVersions( - hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); - metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, - DEFAULT_METRICS_TTL)); - subAppTableDescp.setRegionSplitPolicyClassName( - "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); - subAppTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", - TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); - admin.createTable(subAppTableDescp, - TimelineHBaseSchemaConstants.getUsernameSplits()); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } - - /** - * @param metricsTTL time to live parameter for the metricss in this table. - * @param hbaseConf configururation in which to set the metrics TTL config - * variable. - */ - public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { - hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); - } - -}