|
@@ -61,8 +61,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReader
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
|
|
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.TimelineReader.Field;
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
|
|
|
|
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
|
|
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.entity.EntityTable;
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
|
|
import org.junit.AfterClass;
|
|
import org.junit.AfterClass;
|
|
import org.junit.BeforeClass;
|
|
import org.junit.BeforeClass;
|
|
@@ -94,8 +94,8 @@ public class TestHBaseStorageFlowRun {
|
|
@Test
|
|
@Test
|
|
public void checkCoProcessorOff() throws IOException, InterruptedException {
|
|
public void checkCoProcessorOff() throws IOException, InterruptedException {
|
|
Configuration hbaseConf = util.getConfiguration();
|
|
Configuration hbaseConf = util.getConfiguration();
|
|
- TableName table = TableName.valueOf(hbaseConf.get(
|
|
|
|
- FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
|
|
|
|
|
|
+ TableName table = BaseTable.getTableName(hbaseConf,
|
|
|
|
+ FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME);
|
|
Connection conn = null;
|
|
Connection conn = null;
|
|
conn = ConnectionFactory.createConnection(hbaseConf);
|
|
conn = ConnectionFactory.createConnection(hbaseConf);
|
|
Admin admin = conn.getAdmin();
|
|
Admin admin = conn.getAdmin();
|
|
@@ -109,14 +109,14 @@ public class TestHBaseStorageFlowRun {
|
|
HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
List<Region> regions = server.getOnlineRegions(table);
|
|
List<Region> regions = server.getOnlineRegions(table);
|
|
for (Region region : regions) {
|
|
for (Region region : regions) {
|
|
- assertTrue(HBaseTimelineStorageUtils.isFlowRunTable(
|
|
|
|
- region.getRegionInfo(), hbaseConf));
|
|
|
|
|
|
+ assertTrue(FlowRunTable.isFlowRunTable(region.getRegionInfo(),
|
|
|
|
+ hbaseConf));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- table = TableName.valueOf(hbaseConf.get(
|
|
|
|
|
|
+ table = BaseTable.getTableName(hbaseConf,
|
|
FlowActivityTable.TABLE_NAME_CONF_NAME,
|
|
FlowActivityTable.TABLE_NAME_CONF_NAME,
|
|
- FlowActivityTable.DEFAULT_TABLE_NAME));
|
|
|
|
|
|
+ FlowActivityTable.DEFAULT_TABLE_NAME);
|
|
if (admin.tableExists(table)) {
|
|
if (admin.tableExists(table)) {
|
|
// check the regions.
|
|
// check the regions.
|
|
// check in flow activity table
|
|
// check in flow activity table
|
|
@@ -124,14 +124,13 @@ public class TestHBaseStorageFlowRun {
|
|
HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
List<Region> regions = server.getOnlineRegions(table);
|
|
List<Region> regions = server.getOnlineRegions(table);
|
|
for (Region region : regions) {
|
|
for (Region region : regions) {
|
|
- assertFalse(HBaseTimelineStorageUtils.isFlowRunTable(
|
|
|
|
- region.getRegionInfo(), hbaseConf));
|
|
|
|
|
|
+ assertFalse(FlowRunTable.isFlowRunTable(region.getRegionInfo(),
|
|
|
|
+ hbaseConf));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- table = TableName.valueOf(hbaseConf.get(
|
|
|
|
- EntityTable.TABLE_NAME_CONF_NAME,
|
|
|
|
- EntityTable.DEFAULT_TABLE_NAME));
|
|
|
|
|
|
+ table = BaseTable.getTableName(hbaseConf, EntityTable.TABLE_NAME_CONF_NAME,
|
|
|
|
+ EntityTable.DEFAULT_TABLE_NAME);
|
|
if (admin.tableExists(table)) {
|
|
if (admin.tableExists(table)) {
|
|
// check the regions.
|
|
// check the regions.
|
|
// check in entity run table
|
|
// check in entity run table
|
|
@@ -139,8 +138,8 @@ public class TestHBaseStorageFlowRun {
|
|
HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
List<Region> regions = server.getOnlineRegions(table);
|
|
List<Region> regions = server.getOnlineRegions(table);
|
|
for (Region region : regions) {
|
|
for (Region region : regions) {
|
|
- assertFalse(HBaseTimelineStorageUtils.isFlowRunTable(
|
|
|
|
- region.getRegionInfo(), hbaseConf));
|
|
|
|
|
|
+ assertFalse(FlowRunTable.isFlowRunTable(region.getRegionInfo(),
|
|
|
|
+ hbaseConf));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -220,8 +219,8 @@ public class TestHBaseStorageFlowRun {
|
|
|
|
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
// check in flow run table
|
|
// check in flow run table
|
|
- Table table1 = conn.getTable(TableName
|
|
|
|
- .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
|
|
|
|
|
|
+ Table table1 = conn.getTable(BaseTable.getTableName(c1,
|
|
|
|
+ FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
|
|
// scan the table and see that we get back the right min and max
|
|
// scan the table and see that we get back the right min and max
|
|
// timestamps
|
|
// timestamps
|
|
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
|
|
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
|
|
@@ -356,24 +355,24 @@ public class TestHBaseStorageFlowRun {
|
|
/*
|
|
/*
|
|
* checks the batch limits on a scan
|
|
* checks the batch limits on a scan
|
|
*/
|
|
*/
|
|
- void checkFlowRunTableBatchLimit(String cluster, String user,
|
|
|
|
- String flow, long runid, Configuration c1) throws IOException {
|
|
|
|
|
|
+ void checkFlowRunTableBatchLimit(String cluster, String user, String flow,
|
|
|
|
+ long runid, Configuration c1) throws IOException {
|
|
|
|
|
|
Scan s = new Scan();
|
|
Scan s = new Scan();
|
|
s.addFamily(FlowRunColumnFamily.INFO.getBytes());
|
|
s.addFamily(FlowRunColumnFamily.INFO.getBytes());
|
|
- byte[] startRow =
|
|
|
|
- new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
|
|
|
|
|
|
+ byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid)
|
|
|
|
+ .getRowKey();
|
|
s.setStartRow(startRow);
|
|
s.setStartRow(startRow);
|
|
// set a batch limit
|
|
// set a batch limit
|
|
int batchLimit = 2;
|
|
int batchLimit = 2;
|
|
s.setBatch(batchLimit);
|
|
s.setBatch(batchLimit);
|
|
String clusterStop = cluster + "1";
|
|
String clusterStop = cluster + "1";
|
|
- byte[] stopRow =
|
|
|
|
- new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
|
|
|
|
|
|
+ byte[] stopRow = new FlowRunRowKey(clusterStop, user, flow, runid)
|
|
|
|
+ .getRowKey();
|
|
s.setStopRow(stopRow);
|
|
s.setStopRow(stopRow);
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
- Table table1 = conn
|
|
|
|
- .getTable(TableName.valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
|
|
|
|
|
|
+ Table table1 = conn.getTable(BaseTable.getTableName(c1,
|
|
|
|
+ FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
|
|
ResultScanner scanner = table1.getScanner(s);
|
|
ResultScanner scanner = table1.getScanner(s);
|
|
|
|
|
|
int loopCount = 0;
|
|
int loopCount = 0;
|
|
@@ -517,8 +516,8 @@ public class TestHBaseStorageFlowRun {
|
|
new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
|
|
new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
|
|
s.setStopRow(stopRow);
|
|
s.setStopRow(stopRow);
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
- Table table1 = conn.getTable(TableName
|
|
|
|
- .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
|
|
|
|
|
|
+ Table table1 = conn.getTable(BaseTable.getTableName(c1,
|
|
|
|
+ FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
|
|
ResultScanner scanner = table1.getScanner(s);
|
|
ResultScanner scanner = table1.getScanner(s);
|
|
|
|
|
|
int rowCount = 0;
|
|
int rowCount = 0;
|
|
@@ -782,8 +781,8 @@ public class TestHBaseStorageFlowRun {
|
|
boolean checkMax) throws IOException {
|
|
boolean checkMax) throws IOException {
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
Connection conn = ConnectionFactory.createConnection(c1);
|
|
// check in flow run table
|
|
// check in flow run table
|
|
- Table table1 = conn.getTable(TableName
|
|
|
|
- .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
|
|
|
|
|
|
+ Table table1 = conn.getTable(BaseTable.getTableName(c1,
|
|
|
|
+ FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
|
|
// scan the table and see that we get back the right min and max
|
|
// scan the table and see that we get back the right min and max
|
|
// timestamps
|
|
// timestamps
|
|
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
|
|
byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
|