|
@@ -19,18 +19,21 @@
|
|
|
package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.EnumSet;
|
|
|
+import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|
|
import org.apache.hadoop.hbase.TableName;
|
|
|
+import org.apache.hadoop.hbase.client.Admin;
|
|
|
import org.apache.hadoop.hbase.client.Connection;
|
|
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
|
|
import org.apache.hadoop.hbase.client.Get;
|
|
@@ -38,6 +41,8 @@ import org.apache.hadoop.hbase.client.Result;
|
|
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
|
|
import org.apache.hadoop.hbase.client.Scan;
|
|
|
import org.apache.hadoop.hbase.client.Table;
|
|
|
+import org.apache.hadoop.hbase.regionserver.HRegion;
|
|
|
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
|
|
import org.apache.hadoop.hbase.util.Bytes;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
|
@@ -57,6 +62,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriter
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
|
|
|
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
|
|
|
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
|
|
|
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
|
|
|
import org.junit.AfterClass;
|
|
|
import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
@@ -84,6 +91,60 @@ public class TestHBaseStorageFlowRun {
|
|
|
TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void checkCoProcessorOff() throws IOException, InterruptedException {
|
|
|
+ Configuration hbaseConf = util.getConfiguration();
|
|
|
+ TableName table = TableName.valueOf(hbaseConf.get(
|
|
|
+ FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
|
|
|
+ Connection conn = null;
|
|
|
+ conn = ConnectionFactory.createConnection(hbaseConf);
|
|
|
+ Admin admin = conn.getAdmin();
|
|
|
+ if (admin == null) {
|
|
|
+ throw new IOException("Can't check tables since admin is null");
|
|
|
+ }
|
|
|
+ if (admin.tableExists(table)) {
|
|
|
+ // check the regions.
|
|
|
+ // check in flow run table
|
|
|
+ util.waitUntilAllRegionsAssigned(table);
|
|
|
+ HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
|
+ List<HRegion> regions = server.getOnlineRegions(table);
|
|
|
+ for (HRegion region : regions) {
|
|
|
+ assertTrue(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
|
|
|
+ hbaseConf));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ table = TableName.valueOf(hbaseConf.get(
|
|
|
+ FlowActivityTable.TABLE_NAME_CONF_NAME,
|
|
|
+ FlowActivityTable.DEFAULT_TABLE_NAME));
|
|
|
+ if (admin.tableExists(table)) {
|
|
|
+ // check the regions.
|
|
|
+ // check in flow activity table
|
|
|
+ util.waitUntilAllRegionsAssigned(table);
|
|
|
+ HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
|
+ List<HRegion> regions = server.getOnlineRegions(table);
|
|
|
+ for (HRegion region : regions) {
|
|
|
+ assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
|
|
|
+ hbaseConf));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ table = TableName.valueOf(hbaseConf.get(
|
|
|
+ EntityTable.TABLE_NAME_CONF_NAME,
|
|
|
+ EntityTable.DEFAULT_TABLE_NAME));
|
|
|
+ if (admin.tableExists(table)) {
|
|
|
+ // check the regions.
|
|
|
+ // check in entity run table
|
|
|
+ util.waitUntilAllRegionsAssigned(table);
|
|
|
+ HRegionServer server = util.getRSForFirstRegionInTable(table);
|
|
|
+ List<HRegion> regions = server.getOnlineRegions(table);
|
|
|
+ for (HRegion region : regions) {
|
|
|
+ assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
|
|
|
+ hbaseConf));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Writes 4 timeline entities belonging to one flow run through the
|
|
|
* {@link HBaseTimelineWriterImpl}
|