You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2017/09/08 17:35:50 UTC
[1/5] hbase git commit: HBASE-18106 Redo ProcedureInfo and LockInfo
Repository: hbase
Updated Branches:
refs/heads/master 87c3ae931 -> 359fed7b4
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
index d85146a..e2e97dc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
@@ -18,6 +18,12 @@
package org.apache.hadoop.hbase.master.locking;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -26,10 +32,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.locking.LockProcedure;
-import org.apache.hadoop.hbase.master.locking.TestLockProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -43,12 +48,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
@Category({MasterTests.class, SmallTests.class})
public class TestLockManager {
@Rule
@@ -94,7 +93,7 @@ public class TestLockManager {
@After
public void tearDown() throws Exception {
- for (Procedure<?> proc : getMasterProcedureExecutor().listProcedures()) {
+ for (Procedure<?> proc : getMasterProcedureExecutor().getProcedures()) {
if (proc instanceof LockProcedure) {
((LockProcedure) proc).unlock(getMasterProcedureExecutor().getEnvironment());
ProcedureTestingUtility.waitProcedure(getMasterProcedureExecutor(), proc);
@@ -113,7 +112,7 @@ public class TestLockManager {
@Test
public void testMasterLockAcquire() throws Exception {
LockManager.MasterLock lock = masterServices.getLockManager().createMasterLock(namespace,
- LockProcedure.LockType.EXCLUSIVE, "desc");
+ LockType.EXCLUSIVE, "desc");
assertTrue(lock.tryAcquire(2000));
assertTrue(lock.getProc().isLocked());
lock.release();
@@ -126,9 +125,9 @@ public class TestLockManager {
@Test
public void testMasterLockAcquireTimeout() throws Exception {
LockManager.MasterLock lock = masterServices.getLockManager().createMasterLock(
- tableName, LockProcedure.LockType.EXCLUSIVE, "desc");
+ tableName, LockType.EXCLUSIVE, "desc");
LockManager.MasterLock lock2 = masterServices.getLockManager().createMasterLock(
- tableName, LockProcedure.LockType.EXCLUSIVE, "desc");
+ tableName, LockType.EXCLUSIVE, "desc");
assertTrue(lock.tryAcquire(2000));
assertFalse(lock2.tryAcquire(LOCAL_LOCKS_TIMEOUT/2)); // wait less than other lock's timeout
assertEquals(null, lock2.getProc());
@@ -146,7 +145,7 @@ public class TestLockManager {
LockManager.MasterLock lock = masterServices.getLockManager().createMasterLock(
tableRegions, "desc");
LockManager.MasterLock lock2 = masterServices.getLockManager().createMasterLock(
- tableName, LockProcedure.LockType.EXCLUSIVE, "desc");
+ tableName, LockType.EXCLUSIVE, "desc");
assertTrue(lock.tryAcquire(2000));
assertFalse(lock2.tryAcquire(LOCAL_LOCKS_TIMEOUT/2)); // wait less than other lock's timeout
assertEquals(null, lock2.getProc());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
index adaebf4..e338849 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
@@ -18,9 +18,20 @@
package org.apache.hadoop.hbase.master.locking;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
@@ -28,40 +39,33 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.locking.LockServiceClient;
+import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.MasterRpcServices;
+import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.*;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.hamcrest.core.IsInstanceOf;
import org.hamcrest.core.StringStartsWith;
-import org.junit.rules.TestRule;
-import org.junit.experimental.categories.Category;
-
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
+import org.junit.experimental.categories.Category;
import org.junit.rules.ExpectedException;
import org.junit.rules.TestName;
-import org.apache.hadoop.hbase.CategoryBasedTimeout;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import org.junit.rules.TestRule;
@Category({MasterTests.class, SmallTests.class})
public class TestLockProcedure {
@@ -130,7 +134,7 @@ public class TestLockProcedure {
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
// Kill all running procedures.
- for (Procedure<?> proc : procExec.listProcedures()) {
+ for (Procedure<?> proc : procExec.getProcedures()) {
procExec.abort(proc.getProcId());
ProcedureTestingUtility.waitProcedure(procExec, proc);
}
@@ -138,17 +142,17 @@ public class TestLockProcedure {
}
private LockRequest getNamespaceLock(String namespace, String description) {
- return LockServiceClient.buildLockRequest(LockType.EXCLUSIVE,
+ return LockServiceClient.buildLockRequest(LockServiceProtos.LockType.EXCLUSIVE,
namespace, null, null, description, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
private LockRequest getTableExclusiveLock(TableName tableName, String description) {
- return LockServiceClient.buildLockRequest(LockType.EXCLUSIVE,
+ return LockServiceClient.buildLockRequest(LockServiceProtos.LockType.EXCLUSIVE,
null, tableName, null, description, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
private LockRequest getRegionLock(List<HRegionInfo> regionInfos, String description) {
- return LockServiceClient.buildLockRequest(LockType.EXCLUSIVE,
+ return LockServiceClient.buildLockRequest(LockServiceProtos.LockType.EXCLUSIVE,
null, null, regionInfos, description, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
@@ -345,7 +349,7 @@ public class TestLockProcedure {
CountDownLatch latch = new CountDownLatch(1);
// MasterRpcServices don't set latch with LockProcedure, so create one and submit it directly.
LockProcedure lockProc = new LockProcedure(UTIL.getConfiguration(),
- TableName.valueOf("table"), LockProcedure.LockType.EXCLUSIVE, "desc", latch);
+ TableName.valueOf("table"), org.apache.hadoop.hbase.procedure2.LockType.EXCLUSIVE, "desc", latch);
procExec.submitProcedure(lockProc);
assertTrue(latch.await(2000, TimeUnit.MILLISECONDS));
releaseLock(lockProc.getProcId());
@@ -359,7 +363,7 @@ public class TestLockProcedure {
CountDownLatch latch = new CountDownLatch(1);
// MasterRpcServices don't set latch with LockProcedure, so create one and submit it directly.
LockProcedure lockProc = new LockProcedure(UTIL.getConfiguration(),
- TableName.valueOf("table"), LockProcedure.LockType.EXCLUSIVE, "desc", latch);
+ TableName.valueOf("table"), LockType.EXCLUSIVE, "desc", latch);
procExec.submitProcedure(lockProc);
assertTrue(awaitForLocked(lockProc.getProcId(), 2000));
Thread.sleep(LOCAL_LOCKS_TIMEOUT / 2);
@@ -421,7 +425,7 @@ public class TestLockProcedure {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
CountDownLatch latch = new CountDownLatch(1);
LockProcedure lockProc = new LockProcedure(UTIL.getConfiguration(),
- TableName.valueOf("table"), LockProcedure.LockType.EXCLUSIVE, "desc", latch);
+ TableName.valueOf("table"), LockType.EXCLUSIVE, "desc", latch);
procExec.submitProcedure(lockProc);
assertTrue(latch.await(2000, TimeUnit.MILLISECONDS));
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
index 5f20c7f..e2d6b0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
@@ -32,10 +32,11 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.locking.LockProcedure;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
-import org.apache.hadoop.hbase.procedure2.LockInfo.WaitingProcedure;
+import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
+import org.apache.hadoop.hbase.procedure2.LockedResourceType;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -903,7 +904,7 @@ public class TestMasterProcedureScheduler {
}
}
- private static LockProcedure createLockProcedure(LockProcedure.LockType lockType, long procId) throws Exception {
+ private static LockProcedure createLockProcedure(LockType lockType, long procId) throws Exception {
LockProcedure procedure = new LockProcedure();
Field typeField = LockProcedure.class.getDeclaredField("type");
@@ -918,31 +919,31 @@ public class TestMasterProcedureScheduler {
}
private static LockProcedure createExclusiveLockProcedure(long procId) throws Exception {
- return createLockProcedure(LockProcedure.LockType.EXCLUSIVE, procId);
+ return createLockProcedure(LockType.EXCLUSIVE, procId);
}
private static LockProcedure createSharedLockProcedure(long procId) throws Exception {
- return createLockProcedure(LockProcedure.LockType.SHARED, procId);
+ return createLockProcedure(LockType.SHARED, procId);
}
- private static void assertLockResource(LockInfo lock,
- LockInfo.ResourceType resourceType, String resourceName)
+ private static void assertLockResource(LockedResource resource,
+ LockedResourceType resourceType, String resourceName)
{
- assertEquals(resourceType, lock.getResourceType());
- assertEquals(resourceName, lock.getResourceName());
+ assertEquals(resourceType, resource.getResourceType());
+ assertEquals(resourceName, resource.getResourceName());
}
- private static void assertExclusiveLock(LockInfo lock, long procId)
+ private static void assertExclusiveLock(LockedResource resource, Procedure<?> procedure)
{
- assertEquals(LockInfo.LockType.EXCLUSIVE, lock.getLockType());
- assertEquals(procId, lock.getExclusiveLockOwnerProcedure().getProcId());
- assertEquals(0, lock.getSharedLockCount());
+ assertEquals(LockType.EXCLUSIVE, resource.getLockType());
+ assertEquals(procedure, resource.getExclusiveLockOwnerProcedure());
+ assertEquals(0, resource.getSharedLockCount());
}
- private static void assertSharedLock(LockInfo lock, int lockCount)
+ private static void assertSharedLock(LockedResource resource, int lockCount)
{
- assertEquals(LockInfo.LockType.SHARED, lock.getLockType());
- assertEquals(lockCount, lock.getSharedLockCount());
+ assertEquals(LockType.SHARED, resource.getLockType());
+ assertEquals(lockCount, resource.getSharedLockCount());
}
@Test
@@ -950,13 +951,13 @@ public class TestMasterProcedureScheduler {
LockProcedure procedure = createExclusiveLockProcedure(0);
queue.waitServerExclusiveLock(procedure, ServerName.valueOf("server1,1234,0"));
- List<LockInfo> locks = queue.listLocks();
- assertEquals(1, locks.size());
+ List<LockedResource> resources = queue.getLocks();
+ assertEquals(1, resources.size());
- LockInfo serverLock = locks.get(0);
- assertLockResource(serverLock, LockInfo.ResourceType.SERVER, "server1,1234,0");
- assertExclusiveLock(serverLock, 0);
- assertTrue(serverLock.getWaitingProcedures().isEmpty());
+ LockedResource serverResource = resources.get(0);
+ assertLockResource(serverResource, LockedResourceType.SERVER, "server1,1234,0");
+ assertExclusiveLock(serverResource, procedure);
+ assertTrue(serverResource.getWaitingProcedures().isEmpty());
}
@Test
@@ -964,19 +965,19 @@ public class TestMasterProcedureScheduler {
LockProcedure procedure = createExclusiveLockProcedure(1);
queue.waitNamespaceExclusiveLock(procedure, "ns1");
- List<LockInfo> locks = queue.listLocks();
+ List<LockedResource> locks = queue.getLocks();
assertEquals(2, locks.size());
- LockInfo namespaceLock = locks.get(0);
- assertLockResource(namespaceLock, LockInfo.ResourceType.NAMESPACE, "ns1");
- assertExclusiveLock(namespaceLock, 1);
- assertTrue(namespaceLock.getWaitingProcedures().isEmpty());
+ LockedResource namespaceResource = locks.get(0);
+ assertLockResource(namespaceResource, LockedResourceType.NAMESPACE, "ns1");
+ assertExclusiveLock(namespaceResource, procedure);
+ assertTrue(namespaceResource.getWaitingProcedures().isEmpty());
- LockInfo tableLock = locks.get(1);
- assertLockResource(tableLock, LockInfo.ResourceType.TABLE,
+ LockedResource tableResource = locks.get(1);
+ assertLockResource(tableResource, LockedResourceType.TABLE,
TableName.NAMESPACE_TABLE_NAME.getNameAsString());
- assertSharedLock(tableLock, 1);
- assertTrue(tableLock.getWaitingProcedures().isEmpty());
+ assertSharedLock(tableResource, 1);
+ assertTrue(tableResource.getWaitingProcedures().isEmpty());
}
@Test
@@ -984,18 +985,18 @@ public class TestMasterProcedureScheduler {
LockProcedure procedure = createExclusiveLockProcedure(2);
queue.waitTableExclusiveLock(procedure, TableName.valueOf("ns2", "table2"));
- List<LockInfo> locks = queue.listLocks();
+ List<LockedResource> locks = queue.getLocks();
assertEquals(2, locks.size());
- LockInfo namespaceLock = locks.get(0);
- assertLockResource(namespaceLock, LockInfo.ResourceType.NAMESPACE, "ns2");
- assertSharedLock(namespaceLock, 1);
- assertTrue(namespaceLock.getWaitingProcedures().isEmpty());
+ LockedResource namespaceResource = locks.get(0);
+ assertLockResource(namespaceResource, LockedResourceType.NAMESPACE, "ns2");
+ assertSharedLock(namespaceResource, 1);
+ assertTrue(namespaceResource.getWaitingProcedures().isEmpty());
- LockInfo tableLock = locks.get(1);
- assertLockResource(tableLock, LockInfo.ResourceType.TABLE, "ns2:table2");
- assertExclusiveLock(tableLock, 2);
- assertTrue(tableLock.getWaitingProcedures().isEmpty());
+ LockedResource tableResource = locks.get(1);
+ assertLockResource(tableResource, LockedResourceType.TABLE, "ns2:table2");
+ assertExclusiveLock(tableResource, procedure);
+ assertTrue(tableResource.getWaitingProcedures().isEmpty());
}
@Test
@@ -1005,23 +1006,23 @@ public class TestMasterProcedureScheduler {
queue.waitRegion(procedure, regionInfo);
- List<LockInfo> locks = queue.listLocks();
- assertEquals(3, locks.size());
+ List<LockedResource> resources = queue.getLocks();
+ assertEquals(3, resources.size());
- LockInfo namespaceLock = locks.get(0);
- assertLockResource(namespaceLock, LockInfo.ResourceType.NAMESPACE, "ns3");
- assertSharedLock(namespaceLock, 1);
- assertTrue(namespaceLock.getWaitingProcedures().isEmpty());
+ LockedResource namespaceResource = resources.get(0);
+ assertLockResource(namespaceResource, LockedResourceType.NAMESPACE, "ns3");
+ assertSharedLock(namespaceResource, 1);
+ assertTrue(namespaceResource.getWaitingProcedures().isEmpty());
- LockInfo tableLock = locks.get(1);
- assertLockResource(tableLock, LockInfo.ResourceType.TABLE, "ns3:table3");
- assertSharedLock(tableLock, 1);
- assertTrue(tableLock.getWaitingProcedures().isEmpty());
+ LockedResource tableResource = resources.get(1);
+ assertLockResource(tableResource, LockedResourceType.TABLE, "ns3:table3");
+ assertSharedLock(tableResource, 1);
+ assertTrue(tableResource.getWaitingProcedures().isEmpty());
- LockInfo regionLock = locks.get(2);
- assertLockResource(regionLock, LockInfo.ResourceType.REGION, regionInfo.getEncodedName());
- assertExclusiveLock(regionLock, 3);
- assertTrue(regionLock.getWaitingProcedures().isEmpty());
+ LockedResource regionResource = resources.get(2);
+ assertLockResource(regionResource, LockedResourceType.REGION, regionInfo.getEncodedName());
+ assertExclusiveLock(regionResource, procedure);
+ assertTrue(regionResource.getWaitingProcedures().isEmpty());
}
@Test
@@ -1035,28 +1036,28 @@ public class TestMasterProcedureScheduler {
LockProcedure procedure3 = createExclusiveLockProcedure(3);
queue.waitTableExclusiveLock(procedure3, TableName.valueOf("ns4", "table4"));
- List<LockInfo> locks = queue.listLocks();
- assertEquals(2, locks.size());
+ List<LockedResource> resources = queue.getLocks();
+ assertEquals(2, resources.size());
- LockInfo namespaceLock = locks.get(0);
- assertLockResource(namespaceLock, LockInfo.ResourceType.NAMESPACE, "ns4");
- assertSharedLock(namespaceLock, 1);
- assertTrue(namespaceLock.getWaitingProcedures().isEmpty());
+ LockedResource namespaceResource = resources.get(0);
+ assertLockResource(namespaceResource, LockedResourceType.NAMESPACE, "ns4");
+ assertSharedLock(namespaceResource, 1);
+ assertTrue(namespaceResource.getWaitingProcedures().isEmpty());
- LockInfo tableLock = locks.get(1);
- assertLockResource(tableLock, LockInfo.ResourceType.TABLE, "ns4:table4");
- assertExclusiveLock(tableLock, 1);
+ LockedResource tableLock = resources.get(1);
+ assertLockResource(tableLock, LockedResourceType.TABLE, "ns4:table4");
+ assertExclusiveLock(tableLock, procedure1);
- List<WaitingProcedure> waitingProcedures = tableLock.getWaitingProcedures();
+ List<Procedure<?>> waitingProcedures = tableLock.getWaitingProcedures();
assertEquals(2, waitingProcedures.size());
- WaitingProcedure waitingProcedure1 = waitingProcedures.get(0);
- assertEquals(LockInfo.LockType.SHARED, waitingProcedure1.getLockType());
- assertEquals(2, waitingProcedure1.getProcedure().getProcId());
+ LockProcedure waitingProcedure2 = (LockProcedure) waitingProcedures.get(0);
+ assertEquals(LockType.SHARED, waitingProcedure2.getType());
+ assertEquals(procedure2, waitingProcedure2);
- WaitingProcedure waitingProcedure2 = waitingProcedures.get(1);
- assertEquals(LockInfo.LockType.EXCLUSIVE, waitingProcedure2.getLockType());
- assertEquals(3, waitingProcedure2.getProcedure().getProcId());
+ LockProcedure waitingProcedure3 = (LockProcedure) waitingProcedures.get(1);
+ assertEquals(LockType.EXCLUSIVE, waitingProcedure3.getType());
+ assertEquals(procedure3, waitingProcedure3);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index 692815f..38a12e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -193,7 +193,7 @@ public class TestProcedureAdmin {
}
@Test(timeout=60000)
- public void testListProcedure() throws Exception {
+ public void testGetProcedure() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -206,10 +206,10 @@ public class TestProcedureAdmin {
// Wait for one step to complete
ProcedureTestingUtility.waitProcedure(procExec, procId);
- List<Procedure> listProcedures = procExec.listProcedures();
- assertTrue(listProcedures.size() >= 1);
+ List<Procedure<?>> procedures = procExec.getProcedures();
+ assertTrue(procedures.size() >= 1);
boolean found = false;
- for (Procedure proc: listProcedures) {
+ for (Procedure<?> proc: procedures) {
if (proc.getProcId() == procId) {
assertTrue(proc.isRunnable());
found = true;
@@ -223,8 +223,8 @@ public class TestProcedureAdmin {
ProcedureTestingUtility.restart(procExec);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
- listProcedures = procExec.listProcedures();
- for (Procedure proc: listProcedures) {
+ procedures = procExec.getProcedures();
+ for (Procedure proc: procedures) {
assertTrue(proc.isSuccess());
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureDescriber.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureDescriber.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureDescriber.java
new file mode 100644
index 0000000..1cf33c4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureDescriber.java
@@ -0,0 +1,83 @@
+/*
+ * 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.hbase.procedure;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.Date;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.master.procedure.ProcedureDescriber;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.BytesValue;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, SmallTests.class})
+public class TestProcedureDescriber {
+ public static class TestProcedure extends Procedure {
+ @Override
+ protected Procedure[] execute(Object env) throws ProcedureYieldException,
+ ProcedureSuspendedException, InterruptedException {
+ return null;
+ }
+
+ @Override
+ protected void rollback(Object env)
+ throws IOException, InterruptedException {
+ }
+
+ @Override
+ protected boolean abort(Object env) {
+ return false;
+ }
+
+ @Override
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ ByteString byteString = ByteString.copyFrom(new byte[] { 'A' });
+ BytesValue state = BytesValue.newBuilder().setValue(byteString).build();
+ serializer.serialize(state);
+ }
+
+ @Override
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ }
+ }
+
+ @Test
+ public void test() {
+ TestProcedure procedure = new TestProcedure();
+ String result = ProcedureDescriber.describe(procedure);
+
+ Date epoch = new Date(0);
+
+ assertEquals("{ ID => '-1', PARENT_ID => '-1', STATE => 'INITIALIZING', OWNER => '', "
+ + "TYPE => 'org.apache.hadoop.hbase.procedure.TestProcedureDescriber$TestProcedure', "
+ + "START_TIME => '" + epoch + "', LAST_UPDATE => '" + epoch + "', PARAMETERS => [ "
+ + "{ value => 'QQ==' } ] }", result);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
index 153babf..364055c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.protobuf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -30,15 +28,12 @@ import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.ProcedureState;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
@@ -341,40 +336,4 @@ public class TestProtobufUtil {
Cell newOffheapKV = ProtobufUtil.toCell(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
assertTrue(CellComparator.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
}
-
- private static ProcedureInfo createProcedureInfo(long procId)
- {
- return new ProcedureInfo(procId, "java.lang.Object", null,
- ProcedureState.RUNNABLE, -1, null, null, 0, 0, null);
- }
-
- private static void assertProcedureInfoEquals(ProcedureInfo expected,
- ProcedureInfo result)
- {
- if (expected == result) {
- return;
- } else if (expected == null || result == null) {
- fail();
- }
-
- assertEquals(expected.getProcId(), result.getProcId());
- }
-
- private static void assertLockInfoEquals(LockInfo expected, LockInfo result)
- {
- assertEquals(expected.getResourceType(), result.getResourceType());
- assertEquals(expected.getResourceName(), result.getResourceName());
- assertEquals(expected.getLockType(), result.getLockType());
- assertProcedureInfoEquals(expected.getExclusiveLockOwnerProcedure(),
- result.getExclusiveLockOwnerProcedure());
- assertEquals(expected.getSharedLockCount(), result.getSharedLockCount());
- }
-
- private static void assertWaitingProcedureEquals(
- LockInfo.WaitingProcedure expected, LockInfo.WaitingProcedure result)
- {
- assertEquals(expected.getLockType(), result.getLockType());
- assertProcedureInfoEquals(expected.getProcedure(),
- result.getProcedure());
- }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 1e38179..97b1633 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -32,8 +32,6 @@ import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.Arrays;
@@ -58,7 +56,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
@@ -102,12 +99,12 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.locking.LockProcedure;
-import org.apache.hadoop.hbase.master.locking.LockProcedure.LockType;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
@@ -578,17 +575,19 @@ public class TestAccessController extends SecureTestUtil {
}
@Override
- protected void serializeStateData(OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
TestProcedureProtos.TestTableDDLStateData.Builder testTableDDLMsg =
TestProcedureProtos.TestTableDDLStateData.newBuilder()
.setTableName(tableName.getNameAsString());
- testTableDDLMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(testTableDDLMsg.build());
}
@Override
- protected void deserializeStateData(InputStream stream) throws IOException {
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
TestProcedureProtos.TestTableDDLStateData testTableDDLMsg =
- TestProcedureProtos.TestTableDDLStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(TestProcedureProtos.TestTableDDLStateData.class);
tableName = TableName.valueOf(testTableDDLMsg.getTableName());
}
@@ -630,32 +629,43 @@ public class TestAccessController extends SecureTestUtil {
}
@Test
- public void testListProcedures() throws Exception {
+ public void testGetProcedures() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec =
TEST_UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
Procedure proc = new TestTableDDLProcedure(procExec.getEnvironment(), tableName);
proc.setOwner(USER_OWNER);
procExec.submitProcedure(proc);
- final List<Procedure> procList = procExec.listProcedures();
+ final List<Procedure<?>> procList = procExec.getProcedures();
- AccessTestAction listProceduresAction = new AccessTestAction() {
+ AccessTestAction getProceduresAction = new AccessTestAction() {
@Override
public Object run() throws Exception {
- List<ProcedureInfo> procInfoList = new ArrayList<>(procList.size());
- for(Procedure p : procList) {
- procInfoList.add(ProcedureUtil.convertToProcedureInfo(p));
- }
ACCESS_CONTROLLER
- .postListProcedures(ObserverContext.createAndPrepare(CP_ENV, null), procInfoList);
+ .postGetProcedures(ObserverContext.createAndPrepare(CP_ENV, null), procList);
return null;
}
};
- verifyAllowed(listProceduresAction, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
- verifyAllowed(listProceduresAction, USER_OWNER);
+ verifyAllowed(getProceduresAction, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+ verifyAllowed(getProceduresAction, USER_OWNER);
verifyIfNull(
- listProceduresAction, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
+ getProceduresAction, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
+ }
+
+ @Test (timeout=180000)
+ public void testGetLocks() throws Exception {
+ AccessTestAction action = new AccessTestAction() {
+ @Override
+ public Object run() throws Exception {
+ ACCESS_CONTROLLER.preGetLocks(ObserverContext.createAndPrepare(CP_ENV, null));
+ return null;
+ }
+ };
+
+ verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
+ verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE,
+ USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
}
@Test (timeout=180000)
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
deleted file mode 100644
index c5ad1cc..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
+++ /dev/null
@@ -1,460 +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.hbase.shaded.protobuf;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellBuilderFactory;
-import org.apache.hadoop.hbase.CellBuilderType;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.ProcedureState;
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.DeleteType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos;
-
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-@Category(SmallTests.class)
-public class TestProtobufUtil {
- @Test
- public void testException() throws IOException {
- NameBytesPair.Builder builder = NameBytesPair.newBuilder();
- final String omg = "OMG!!!";
- builder.setName("java.io.IOException");
- builder.setValue(ByteString.copyFrom(Bytes.toBytes(omg)));
- Throwable t = ProtobufUtil.toException(builder.build());
- assertEquals(omg, t.getMessage());
- builder.clear();
- builder.setName("org.apache.hadoop.ipc.RemoteException");
- builder.setValue(ByteString.copyFrom(Bytes.toBytes(omg)));
- t = ProtobufUtil.toException(builder.build());
- assertEquals(omg, t.getMessage());
- }
-
- /**
- * Test basic Get conversions.
- *
- * @throws IOException
- */
- @Test
- public void testGet() throws IOException {
- ClientProtos.Get.Builder getBuilder = ClientProtos.Get.newBuilder();
- getBuilder.setRow(ByteString.copyFromUtf8("row"));
- Column.Builder columnBuilder = Column.newBuilder();
- columnBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- columnBuilder.addQualifier(ByteString.copyFromUtf8("c1"));
- columnBuilder.addQualifier(ByteString.copyFromUtf8("c2"));
- getBuilder.addColumn(columnBuilder.build());
-
- columnBuilder.clear();
- columnBuilder.setFamily(ByteString.copyFromUtf8("f2"));
- getBuilder.addColumn(columnBuilder.build());
- getBuilder.setLoadColumnFamiliesOnDemand(true);
- ClientProtos.Get proto = getBuilder.build();
- // default fields
- assertEquals(1, proto.getMaxVersions());
- assertEquals(true, proto.getCacheBlocks());
-
- // set the default value for equal comparison
- getBuilder = ClientProtos.Get.newBuilder(proto);
- getBuilder.setMaxVersions(1);
- getBuilder.setCacheBlocks(true);
-
- Get get = ProtobufUtil.toGet(proto);
- assertEquals(getBuilder.build(), ProtobufUtil.toGet(get));
- }
-
- /**
- * Test Delete Mutate conversions.
- *
- * @throws IOException
- */
- @Test
- public void testDelete() throws IOException {
- MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
- mutateBuilder.setMutateType(MutationType.DELETE);
- mutateBuilder.setTimestamp(111111);
- ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
- valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
- qualifierBuilder.setDeleteType(DeleteType.DELETE_ONE_VERSION);
- qualifierBuilder.setTimestamp(111222);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
- qualifierBuilder.setDeleteType(DeleteType.DELETE_MULTIPLE_VERSIONS);
- qualifierBuilder.setTimestamp(111333);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- mutateBuilder.addColumnValue(valueBuilder.build());
-
- MutationProto proto = mutateBuilder.build();
- // default fields
- assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
-
- // set the default value for equal comparison
- mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
-
- Delete delete = ProtobufUtil.toDelete(proto);
-
- // delete always have empty value,
- // add empty value to the original mutate
- for (ColumnValue.Builder column:
- mutateBuilder.getColumnValueBuilderList()) {
- for (QualifierValue.Builder qualifier:
- column.getQualifierValueBuilderList()) {
- qualifier.setValue(ByteString.EMPTY);
- }
- }
- assertEquals(mutateBuilder.build(),
- ProtobufUtil.toMutation(MutationType.DELETE, delete));
- }
-
- /**
- * Test Put Mutate conversions.
- *
- * @throws IOException
- */
- @Test
- public void testPut() throws IOException {
- MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
- mutateBuilder.setMutateType(MutationType.PUT);
- mutateBuilder.setTimestamp(111111);
- ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
- valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
- qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
- qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
- qualifierBuilder.setTimestamp(222222);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- mutateBuilder.addColumnValue(valueBuilder.build());
-
- MutationProto proto = mutateBuilder.build();
- // default fields
- assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
-
- // set the default value for equal comparison
- mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
-
- Put put = ProtobufUtil.toPut(proto);
-
- // put value always use the default timestamp if no
- // value level timestamp specified,
- // add the timestamp to the original mutate
- long timestamp = put.getTimeStamp();
- for (ColumnValue.Builder column:
- mutateBuilder.getColumnValueBuilderList()) {
- for (QualifierValue.Builder qualifier:
- column.getQualifierValueBuilderList()) {
- if (!qualifier.hasTimestamp()) {
- qualifier.setTimestamp(timestamp);
- }
- }
- }
- assertEquals(mutateBuilder.build(),
- ProtobufUtil.toMutation(MutationType.PUT, put));
- }
-
- /**
- * Test basic Scan conversions.
- *
- * @throws IOException
- */
- @Test
- public void testScan() throws IOException {
- ClientProtos.Scan.Builder scanBuilder = ClientProtos.Scan.newBuilder();
- scanBuilder.setStartRow(ByteString.copyFromUtf8("row1"));
- scanBuilder.setStopRow(ByteString.copyFromUtf8("row2"));
- Column.Builder columnBuilder = Column.newBuilder();
- columnBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- columnBuilder.addQualifier(ByteString.copyFromUtf8("c1"));
- columnBuilder.addQualifier(ByteString.copyFromUtf8("c2"));
- scanBuilder.addColumn(columnBuilder.build());
-
- columnBuilder.clear();
- columnBuilder.setFamily(ByteString.copyFromUtf8("f2"));
- scanBuilder.addColumn(columnBuilder.build());
-
- ClientProtos.Scan proto = scanBuilder.build();
-
- // Verify default values
- assertEquals(1, proto.getMaxVersions());
- assertEquals(true, proto.getCacheBlocks());
-
- // Verify fields survive ClientProtos.Scan -> Scan -> ClientProtos.Scan
- // conversion
- scanBuilder = ClientProtos.Scan.newBuilder(proto);
- scanBuilder.setMaxVersions(2);
- scanBuilder.setCacheBlocks(false);
- scanBuilder.setCaching(1024);
- ClientProtos.Scan expectedProto = scanBuilder.build();
-
- ClientProtos.Scan actualProto = ProtobufUtil.toScan(
- ProtobufUtil.toScan(expectedProto));
- assertEquals(expectedProto, actualProto);
- }
-
- @Test
- public void testToCell() throws Exception {
- KeyValue kv1 =
- new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
- KeyValue kv2 =
- new KeyValue(Bytes.toBytes("bbb"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
- KeyValue kv3 =
- new KeyValue(Bytes.toBytes("ccc"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
- byte[] arr = new byte[kv1.getLength() + kv2.getLength() + kv3.getLength()];
- System.arraycopy(kv1.getBuffer(), kv1.getOffset(), arr, 0, kv1.getLength());
- System.arraycopy(kv2.getBuffer(), kv2.getOffset(), arr, kv1.getLength(), kv2.getLength());
- System.arraycopy(kv3.getBuffer(), kv3.getOffset(), arr, kv1.getLength() + kv2.getLength(),
- kv3.getLength());
- ByteBuffer dbb = ByteBuffer.allocateDirect(arr.length);
- dbb.put(arr);
- ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(dbb, kv1.getLength(), kv2.getLength());
- CellProtos.Cell cell = ProtobufUtil.toCell(offheapKV);
- Cell newOffheapKV = ProtobufUtil.toCell(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
- assertTrue(CellComparator.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
- }
-
- public TestProtobufUtil() {
- }
-
- private static ProcedureInfo createProcedureInfo(long procId)
- {
- return new ProcedureInfo(procId, "java.lang.Object", null,
- ProcedureState.RUNNABLE, -1, null, null, 0, 0, null);
- }
-
- private static void assertProcedureInfoEquals(ProcedureInfo expected,
- ProcedureInfo result)
- {
- if (expected == result) {
- return;
- } else if (expected == null || result == null) {
- fail();
- }
-
- assertEquals(expected.getProcId(), result.getProcId());
- }
-
- private static void assertLockInfoEquals(LockInfo expected, LockInfo result)
- {
- assertEquals(expected.getResourceType(), result.getResourceType());
- assertEquals(expected.getResourceName(), result.getResourceName());
- assertEquals(expected.getLockType(), result.getLockType());
- assertProcedureInfoEquals(expected.getExclusiveLockOwnerProcedure(),
- result.getExclusiveLockOwnerProcedure());
- assertEquals(expected.getSharedLockCount(), result.getSharedLockCount());
- }
-
- private static void assertWaitingProcedureEquals(
- LockInfo.WaitingProcedure expected, LockInfo.WaitingProcedure result)
- {
- assertEquals(expected.getLockType(), result.getLockType());
- assertProcedureInfoEquals(expected.getProcedure(),
- result.getProcedure());
- }
-
- @Test
- public void testServerLockInfo() {
- LockInfo lock = new LockInfo();
- lock.setResourceType(LockInfo.ResourceType.SERVER);
- lock.setResourceName("server");
- lock.setLockType(LockInfo.LockType.SHARED);
- lock.setSharedLockCount(2);
-
- LockServiceProtos.LockInfo proto = ProtobufUtil.toProtoLockInfo(lock);
- LockInfo lock2 = ProtobufUtil.toLockInfo(proto);
-
- assertLockInfoEquals(lock, lock2);
- }
-
- @Test
- public void testNamespaceLockInfo() {
- LockInfo lock = new LockInfo();
- lock.setResourceType(LockInfo.ResourceType.NAMESPACE);
- lock.setResourceName("ns");
- lock.setLockType(LockInfo.LockType.EXCLUSIVE);
- lock.setExclusiveLockOwnerProcedure(createProcedureInfo(2));
-
- LockServiceProtos.LockInfo proto = ProtobufUtil.toProtoLockInfo(lock);
- LockInfo lock2 = ProtobufUtil.toLockInfo(proto);
-
- assertLockInfoEquals(lock, lock2);
- }
-
- @Test
- public void testTableLockInfo() {
- LockInfo lock = new LockInfo();
- lock.setResourceType(LockInfo.ResourceType.TABLE);
- lock.setResourceName("table");
- lock.setLockType(LockInfo.LockType.SHARED);
- lock.setSharedLockCount(2);
-
- LockServiceProtos.LockInfo proto = ProtobufUtil.toProtoLockInfo(lock);
- LockInfo lock2 = ProtobufUtil.toLockInfo(proto);
-
- assertLockInfoEquals(lock, lock2);
- }
-
- @Test
- public void testRegionLockInfo() {
- LockInfo lock = new LockInfo();
- lock.setResourceType(LockInfo.ResourceType.REGION);
- lock.setResourceName("region");
- lock.setLockType(LockInfo.LockType.EXCLUSIVE);
- lock.setExclusiveLockOwnerProcedure(createProcedureInfo(2));
-
- LockServiceProtos.LockInfo proto = ProtobufUtil.toProtoLockInfo(lock);
- LockInfo lock2 = ProtobufUtil.toLockInfo(proto);
-
- assertLockInfoEquals(lock, lock2);
- }
-
- @Test
- public void testExclusiveWaitingLockInfo() {
- LockInfo.WaitingProcedure waitingProcedure = new LockInfo.WaitingProcedure();
- waitingProcedure.setLockType(LockInfo.LockType.EXCLUSIVE);
- waitingProcedure.setProcedure(createProcedureInfo(1));
-
- LockServiceProtos.WaitingProcedure proto = ProtobufUtil.toProtoWaitingProcedure(waitingProcedure);
- LockInfo.WaitingProcedure waitingProcedure2 = ProtobufUtil.toWaitingProcedure(proto);
-
- assertWaitingProcedureEquals(waitingProcedure, waitingProcedure2);
- }
-
- @Test
- public void testSharedWaitingLockInfo() {
- LockInfo.WaitingProcedure waitingProcedure = new LockInfo.WaitingProcedure();
- waitingProcedure.setLockType(LockInfo.LockType.SHARED);
- waitingProcedure.setProcedure(createProcedureInfo(2));
-
- LockServiceProtos.WaitingProcedure proto = ProtobufUtil.toProtoWaitingProcedure(waitingProcedure);
- LockInfo.WaitingProcedure waitingProcedure2 = ProtobufUtil.toWaitingProcedure(proto);
-
- assertWaitingProcedureEquals(waitingProcedure, waitingProcedure2);
- }
-
- /**
- * Test Increment Mutate conversions.
- *
- * @throws IOException
- */
- @Test
- public void testIncrement() throws IOException {
- long timeStamp = 111111;
- MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
- mutateBuilder.setMutateType(MutationProto.MutationType.INCREMENT);
- ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
- valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
- qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(11L)));
- qualifierBuilder.setTimestamp(timeStamp);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
- qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(22L)));
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- mutateBuilder.addColumnValue(valueBuilder.build());
-
- MutationProto proto = mutateBuilder.build();
- // default fields
- assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
-
- // set the default value for equal comparison
- mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
-
- Increment increment = ProtobufUtil.toIncrement(proto, null);
- mutateBuilder.setTimestamp(increment.getTimeStamp());
- assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
- }
-
- /**
- * Test Append Mutate conversions.
- *
- * @throws IOException
- */
- @Test
- public void testAppend() throws IOException {
- long timeStamp = 111111;
- MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
- mutateBuilder.setMutateType(MutationType.APPEND);
- mutateBuilder.setTimestamp(timeStamp);
- ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
- valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
- QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
- qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
- qualifierBuilder.setTimestamp(timeStamp);
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
- qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
- valueBuilder.addQualifierValue(qualifierBuilder.build());
- mutateBuilder.addColumnValue(valueBuilder.build());
-
- MutationProto proto = mutateBuilder.build();
- // default fields
- assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
-
- // set the default value for equal comparison
- mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
-
- Append append = ProtobufUtil.toAppend(proto, null);
-
- // append always use the latest timestamp,
- // reset the timestamp to the original mutate
- mutateBuilder.setTimestamp(append.getTimeStamp());
- assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 2aacd7f..1dfa0c1 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -1198,12 +1198,12 @@ module Hbase
# List all procedures
def list_procedures
- @admin.listProcedures
+ @admin.getProcedures
end
# List all locks
def list_locks
- @admin.listLocks
+ @admin.getLocks
end
# Parse arguments and update HTableDescriptor accordingly
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-shell/src/main/ruby/shell/commands/list_locks.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_locks.rb b/hbase-shell/src/main/ruby/shell/commands/list_locks.rb
index a7f7b73..1bebdd5 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_locks.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_locks.rb
@@ -17,6 +17,8 @@
# limitations under the License.
#
+require 'json'
+
module Shell
module Commands
class ListLocks < Command
@@ -29,27 +31,28 @@ EOF
end
def command
- list = admin.list_locks
+ list = JSON.parse(admin.list_locks)
list.each do |lock|
- formatter.output_strln("#{lock.resourceType}(#{lock.resourceName})")
-
- case lock.lockType
- when org.apache.hadoop.hbase.procedure2.LockInfo::LockType::EXCLUSIVE then
- formatter.output_strln("Lock type: EXCLUSIVE, procedure: #{lock.exclusiveLockOwnerProcedure.procId}")
- when org.apache.hadoop.hbase.procedure2.LockInfo::LockType::SHARED then
- formatter.output_strln("Lock type: SHARED, count: #{lock.sharedLockCount}")
+ formatter.output_strln("#{lock['resourceType']}(#{lock['resourceName']})")
+
+ case lock['lockType']
+ when 'EXCLUSIVE' then
+ formatter.output_strln("Lock type: #{lock['lockType']}, " \
+ "procedure: #{lock['exclusiveLockOwnerProcedure']}")
+ when 'SHARED' then
+ formatter.output_strln("Lock type: #{lock['lockType']}, " \
+ "count: #{lock['sharedLockCount']}")
end
- if lock.waitingProcedures.any?
- formatter.output_strln('Waiting procedures:')
- formatter.header(['Lock type', 'Procedure Id'])
+ if lock['waitingProcedures']
+ formatter.header(['Waiting procedures'])
- lock.waitingProcedures.each do |waitingProcedure|
- formatter.row([waitingProcedure.lockType.to_s, waitingProcedure.procedure.procId.to_s])
+ lock['waitingProcedures'].each do |waiting_procedure|
+ formatter.row([waiting_procedure])
end
- formatter.footer(lock.waitingProcedures.size)
+ formatter.footer(lock['waitingProcedures'].size)
end
formatter.output_strln('')
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb b/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
index a2bec37..77335b8 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
@@ -17,6 +17,8 @@
# limitations under the License.
#
+require 'json'
+
module Shell
module Commands
class ListProcedures < Command
@@ -29,13 +31,15 @@ EOF
end
def command
- formatter.header(%w[Id Name State Submitted_Time Last_Update])
+ formatter.header(%w[Id Name State Submitted_Time Last_Update Parameters])
- list = admin.list_procedures
+ list = JSON.parse(admin.list_procedures)
list.each do |proc|
- submitted_time = Time.at(proc.getSubmittedTime / 1000).to_s
- last_update = Time.at(proc.getLastUpdate / 1000).to_s
- formatter.row([proc.getProcId, proc.getProcName, proc.getProcState, submitted_time, last_update])
+ formatter.row([proc])
+ submitted_time = Time.at(Integer(proc['submittedTime']) / 1000).to_s
+ last_update = Time.at(Integer(proc['lastUpdate']) / 1000).to_s
+ formatter.row([proc['procId'], proc['className'], proc['state'],
+ submitted_time, last_update, proc['stateMessage']])
end
formatter.footer(list.size)
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
index 4279d89..645119a 100644
--- a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
@@ -33,4 +33,4 @@ public class TestReplicationShell extends AbstractTestShell {
// Start all ruby tests
jruby.runScriptlet(PathType.ABSOLUTE, "src/test/ruby/tests_runner.rb");
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/procedure/ShellTestProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/procedure/ShellTestProcedure.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/procedure/ShellTestProcedure.java
new file mode 100644
index 0000000..742fd91
--- /dev/null
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/procedure/ShellTestProcedure.java
@@ -0,0 +1,87 @@
+/**
+ * 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.hbase.client.procedure;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.StringValue;
+
+public class ShellTestProcedure extends Procedure<Object> implements TableProcedureInterface {
+ private String tableNameString;
+
+ public ShellTestProcedure() {
+ }
+
+ public ShellTestProcedure(String tableNameString) {
+ setTableNameString(tableNameString);
+ }
+
+ public String getTableNameString() {
+ return tableNameString;
+ }
+
+ public void setTableNameString(String tableNameString) {
+ this.tableNameString = tableNameString;
+ }
+
+ @Override
+ public TableName getTableName() {
+ return TableName.valueOf(tableNameString);
+ }
+
+ @Override
+ public TableOperationType getTableOperationType() {
+ return TableOperationType.EDIT;
+ }
+
+ @Override
+ protected Procedure<Object>[] execute(Object env)
+ throws ProcedureYieldException, ProcedureSuspendedException,
+ InterruptedException {
+ return null;
+ }
+
+ @Override
+ protected void rollback(Object env) throws IOException, InterruptedException {
+ }
+
+ @Override
+ protected boolean abort(Object env) {
+ return false;
+ }
+
+ @Override
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ StringValue message = StringValue.newBuilder().setValue(tableNameString).build();
+ serializer.serialize(message);
+ }
+
+ @Override
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ StringValue message = serializer.deserialize(StringValue.class);
+ tableNameString = message.getValue();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/list_locks_test.rb b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
index fe132db..f465a6b 100644
--- a/hbase-shell/src/test/ruby/shell/list_locks_test.rb
+++ b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
@@ -20,133 +20,175 @@
require 'hbase_constants'
require 'shell'
-class ListLocksTest < Test::Unit::TestCase
- def setup
- @hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
- @shell = Shell::Shell.new(@hbase)
- @master = $TEST_CLUSTER.getHBaseClusterInterface.getMaster
- @scheduler = @master.getMasterProcedureExecutor.getEnvironment.getProcedureScheduler
-
- @string_io = StringIO.new
-
- @list_locks = Shell::Commands::ListLocks.new(@shell)
- @list_locks.set_formatter(Shell::Formatter::Base.new({ :output_stream => @string_io }))
- end
-
- def set_field(object, field_name, value)
- field = object.getClass.getDeclaredField(field_name)
- field.setAccessible(true)
- field.set(object, value)
- end
-
- def create_lock(type, proc_id)
- lock = org.apache.hadoop.hbase.master.locking.LockProcedure.new()
- set_field(lock, "type", type)
- lock.procId = proc_id
-
- return lock
- end
-
- def create_exclusive_lock(proc_id)
- return create_lock(org.apache.hadoop.hbase.master.locking.LockProcedure::LockType::EXCLUSIVE, proc_id)
+module Hbase
+ class ListLocksTest < Test::Unit::TestCase
+ include TestHelpers
+
+ def setup
+ setup_hbase
+
+ @master = $TEST_CLUSTER.getHBaseClusterInterface.getMaster
+ @scheduler = @master.getMasterProcedureExecutor.getEnvironment.getProcedureScheduler
+
+ @list_locks = Shell::Commands::ListLocks.new(@shell)
+ end
+
+ def teardown
+ shutdown
+ end
+
+ def set_field(object, field_name, value)
+ field = object.getClass.getDeclaredField(field_name)
+ field.setAccessible(true)
+ field.set(object, value)
+ end
+
+ def create_lock(type, op_type, proc_id)
+ lock = org.apache.hadoop.hbase.master.locking.LockProcedure.new
+ set_field(lock, 'type', type)
+ set_field(lock, 'opType', op_type)
+ set_field(lock, 'description', 'description')
+ lock.procId = proc_id
+ lock.submittedTime = 0
+ lock.lastUpdate = 0
+
+ lock
+ end
+
+ def create_exclusive_lock(proc_id)
+ create_lock(org.apache.hadoop.hbase.procedure2.LockType::EXCLUSIVE,
+ org.apache.hadoop.hbase.master.procedure.TableProcedureInterface::TableOperationType::EDIT,
+ proc_id)
+ end
+
+ def create_shared_lock(proc_id)
+ create_lock(org.apache.hadoop.hbase.procedure2.LockType::SHARED,
+ org.apache.hadoop.hbase.master.procedure.TableProcedureInterface::TableOperationType::READ,
+ proc_id)
+ end
+
+ define_test 'list server locks' do
+ lock = create_exclusive_lock(0)
+
+ server_name = org.apache.hadoop.hbase.ServerName.valueOf('server1,1234,0')
+
+ @scheduler.waitServerExclusiveLock(lock, server_name)
+ output = capture_stdout { @list_locks.command }
+ @scheduler.wakeServerExclusiveLock(lock, server_name)
+
+ assert_equal(
+ "SERVER(server1,1234,0)\n" \
+ "Lock type: EXCLUSIVE, procedure: {" \
+ "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+ "\"procId\"=>\"0\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \
+ "\"lastUpdate\"=>\"0\", " \
+ "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \
+ "}\n\n",
+ output)
+ end
+
+ define_test 'list namespace locks' do
+ lock = create_exclusive_lock(1)
+
+ @scheduler.waitNamespaceExclusiveLock(lock, 'ns1')
+ output = capture_stdout { @list_locks.command }
+ @scheduler.wakeNamespaceExclusiveLock(lock, 'ns1')
+
+ assert_equal(
+ "NAMESPACE(ns1)\n" \
+ "Lock type: EXCLUSIVE, procedure: {" \
+ "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+ "\"procId\"=>\"1\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \
+ "\"lastUpdate\"=>\"0\", " \
+ "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \
+ "}\n\n" \
+ "TABLE(hbase:namespace)\n" \
+ "Lock type: SHARED, count: 1\n\n",
+ output)
+ end
+
+ define_test 'list table locks' do
+ lock = create_exclusive_lock(2)
+
+ table_name = org.apache.hadoop.hbase.TableName.valueOf('ns2', 'table2')
+
+ @scheduler.waitTableExclusiveLock(lock, table_name)
+ output = capture_stdout { @list_locks.command }
+ @scheduler.wakeTableExclusiveLock(lock, table_name)
+
+ assert_equal(
+ "NAMESPACE(ns2)\n" \
+ "Lock type: SHARED, count: 1\n\n" \
+ "TABLE(ns2:table2)\n" \
+ "Lock type: EXCLUSIVE, procedure: {" \
+ "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+ "\"procId\"=>\"2\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \
+ "\"lastUpdate\"=>\"0\", " \
+ "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \
+ "}\n\n",
+ output)
+ end
+
+ define_test 'list region locks' do
+ lock = create_exclusive_lock(3)
+
+ table_name = org.apache.hadoop.hbase.TableName.valueOf('ns3', 'table3')
+ region_info = org.apache.hadoop.hbase.HRegionInfo.new(table_name)
+
+ @scheduler.waitRegion(lock, region_info)
+ output = capture_stdout { @list_locks.command }
+ @scheduler.wakeRegion(lock, region_info)
+
+ assert_equal(
+ "NAMESPACE(ns3)\n" \
+ "Lock type: SHARED, count: 1\n\n" \
+ "TABLE(ns3:table3)\n" \
+ "Lock type: SHARED, count: 1\n\n" \
+ "REGION(" << region_info.getEncodedName << ")\n" \
+ "Lock type: EXCLUSIVE, procedure: {" \
+ "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+ "\"procId\"=>\"3\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \
+ "\"lastUpdate\"=>\"0\", " \
+ "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \
+ "}\n\n",
+ output)
+ end
+
+ define_test 'list waiting locks' do
+ table_name = org.apache.hadoop.hbase.TableName.valueOf('ns4', 'table4')
+
+ lock1 = create_exclusive_lock(1)
+ set_field(lock1, 'tableName', table_name)
+
+ lock2 = create_shared_lock(2)
+ set_field(lock2, 'tableName', table_name)
+
+ @scheduler.waitTableExclusiveLock(lock1, table_name)
+ @scheduler.waitTableSharedLock(lock2, table_name)
+ output = capture_stdout { @list_locks.command }
+ @scheduler.wakeTableExclusiveLock(lock1, table_name)
+ @scheduler.wakeTableSharedLock(lock2, table_name)
+
+ assert_equal(
+ "NAMESPACE(ns4)\n" \
+ "Lock type: SHARED, count: 1\n\n" \
+ "TABLE(ns4:table4)\n" \
+ "Lock type: EXCLUSIVE, procedure: {" \
+ "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+ "\"procId\"=>\"1\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \
+ "\"lastUpdate\"=>\"0\", \"stateMessage\"=>[{" \
+ "\"lockType\"=>\"EXCLUSIVE\", " \
+ "\"tableName\"=>{\"namespace\"=>\"bnM0\", \"qualifier\"=>\"dGFibGU0\"" \
+ "}, \"description\"=>\"description\"}]}\n" \
+ "Waiting procedures\n" \
+ "{\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+ "\"procId\"=>\"2\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \
+ "\"lastUpdate\"=>\"0\", \"stateMessage\"=>[{" \
+ "\"lockType\"=>\"SHARED\", " \
+ "\"tableName\"=>{\"namespace\"=>\"bnM0\", \"qualifier\"=>\"dGFibGU0\"}, " \
+ "\"description\"=>\"description\"}]}\n" \
+ "1 row(s)\n\n",
+ output)
+ end
end
-
- def create_shared_lock(proc_id)
- return create_lock(org.apache.hadoop.hbase.master.locking.LockProcedure::LockType::SHARED, proc_id)
- end
-
- define_test "list server locks" do
- lock = create_exclusive_lock(0)
-
- server_name = org.apache.hadoop.hbase.ServerName.valueOf("server1,1234,0")
-
- @scheduler.waitServerExclusiveLock(lock, server_name)
- @list_locks.command()
- @scheduler.wakeServerExclusiveLock(lock, server_name)
-
- assert_equal(
- "SERVER(server1,1234,0)\n" <<
- "Lock type: EXCLUSIVE, procedure: 0\n\n",
- @string_io.string)
- end
-
- define_test "list namespace locks" do
- lock = create_exclusive_lock(1)
-
- @scheduler.waitNamespaceExclusiveLock(lock, "ns1")
- @list_locks.command()
- @scheduler.wakeNamespaceExclusiveLock(lock, "ns1")
-
- assert_equal(
- "NAMESPACE(ns1)\n" <<
- "Lock type: EXCLUSIVE, procedure: 1\n\n" <<
- "TABLE(hbase:namespace)\n" <<
- "Lock type: SHARED, count: 1\n\n",
- @string_io.string)
- end
-
- define_test "list table locks" do
- lock = create_exclusive_lock(2)
-
- table_name = org.apache.hadoop.hbase.TableName.valueOf("ns2", "table2")
-
- @scheduler.waitTableExclusiveLock(lock, table_name)
- @list_locks.command()
- @scheduler.wakeTableExclusiveLock(lock, table_name)
-
- assert_equal(
- "NAMESPACE(ns2)\n" <<
- "Lock type: SHARED, count: 1\n\n" <<
- "TABLE(ns2:table2)\n" <<
- "Lock type: EXCLUSIVE, procedure: 2\n\n",
- @string_io.string)
- end
-
- define_test "list region locks" do
- lock = create_exclusive_lock(3)
-
- table_name = org.apache.hadoop.hbase.TableName.valueOf("ns3", "table3")
- region_info = org.apache.hadoop.hbase.HRegionInfo.new(table_name)
-
- @scheduler.waitRegion(lock, region_info)
- @list_locks.command()
- @scheduler.wakeRegion(lock, region_info)
-
- assert_equal(
- "NAMESPACE(ns3)\n" <<
- "Lock type: SHARED, count: 1\n\n" <<
- "TABLE(ns3:table3)\n" <<
- "Lock type: SHARED, count: 1\n\n" <<
- "REGION(" << region_info.getEncodedName << ")\n" <<
- "Lock type: EXCLUSIVE, procedure: 3\n\n",
- @string_io.string)
- end
-
- define_test "list waiting locks" do
- table_name = org.apache.hadoop.hbase.TableName.valueOf("ns4", "table4")
-
- lock1 = create_exclusive_lock(1)
- set_field(lock1, "tableName", table_name)
-
- lock2 = create_shared_lock(2)
- set_field(lock2, "tableName", table_name)
-
- @scheduler.waitTableExclusiveLock(lock1, table_name)
- @scheduler.waitTableSharedLock(lock2, table_name)
- @list_locks.command()
- @scheduler.wakeTableExclusiveLock(lock1, table_name)
- @scheduler.wakeTableSharedLock(lock2, table_name)
-
- assert_equal(
- "NAMESPACE(ns4)\n" <<
- "Lock type: SHARED, count: 1\n\n" <<
- "TABLE(ns4:table4)\n" <<
- "Lock type: EXCLUSIVE, procedure: 1\n" <<
- "Waiting procedures:\n" <<
- "Lock type Procedure Id\n" <<
- " SHARED 2\n" <<
- "1 row(s)\n\n",
- @string_io.string)
- end
-
end
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-shell/src/test/ruby/shell/list_procedures_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/list_procedures_test.rb b/hbase-shell/src/test/ruby/shell/list_procedures_test.rb
new file mode 100644
index 0000000..8d5c83d
--- /dev/null
+++ b/hbase-shell/src/test/ruby/shell/list_procedures_test.rb
@@ -0,0 +1,68 @@
+#
+#
+# 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.
+#
+
+require 'hbase_constants'
+require 'shell'
+
+include HBaseConstants
+
+module Hbase
+ class ListProceduresTest < Test::Unit::TestCase
+ include TestHelpers
+
+ def setup
+ setup_hbase
+
+ @master = $TEST_CLUSTER.getHBaseClusterInterface.getMaster
+ @executor = @master.getMasterProcedureExecutor
+
+ @list_procedures = Shell::Commands::ListProcedures.new(@shell)
+ end
+
+ def teardown
+ shutdown
+ end
+
+ def create_procedure_regexp(table_name)
+ regexp_string = '[0-9]+ .*ShellTestProcedure SUCCESS.*' \
+ "\[{\"value\"=>\"#{table_name}\"}\]";
+ Regexp.new(regexp_string)
+ end
+
+ define_test 'list procedures' do
+ procedure = org.apache.hadoop.hbase.client.procedure.ShellTestProcedure.new
+ procedure.tableNameString = 'table1'
+
+ @executor.submitProcedure(procedure)
+ output = capture_stdout { @list_procedures.command }
+
+ regexp = create_procedure_regexp('table1')
+ matching_lines = 0
+
+ lines = output.split(/\n/)
+ lines.each do |line|
+ if regexp.match(line)
+ matching_lines += 1
+ end
+ end
+
+ assert_equal(1, matching_lines)
+ end
+ end
+end
[4/5] hbase git commit: HBASE-18106 Redo ProcedureInfo and LockInfo
Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
new file mode 100644
index 0000000..0c12648
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
@@ -0,0 +1,466 @@
+/**
+ * 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.hbase.shaded.protobuf;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Any;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.BytesValue;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.DeleteType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestProtobufUtil {
+ public TestProtobufUtil() {
+ }
+
+ @Test
+ public void testException() throws IOException {
+ NameBytesPair.Builder builder = NameBytesPair.newBuilder();
+ final String omg = "OMG!!!";
+ builder.setName("java.io.IOException");
+ builder.setValue(ByteString.copyFrom(Bytes.toBytes(omg)));
+ Throwable t = ProtobufUtil.toException(builder.build());
+ assertEquals(omg, t.getMessage());
+ builder.clear();
+ builder.setName("org.apache.hadoop.ipc.RemoteException");
+ builder.setValue(ByteString.copyFrom(Bytes.toBytes(omg)));
+ t = ProtobufUtil.toException(builder.build());
+ assertEquals(omg, t.getMessage());
+ }
+
+ /**
+ * Test basic Get conversions.
+ *
+ * @throws IOException
+ */
+ @Test
+ public void testGet() throws IOException {
+ ClientProtos.Get.Builder getBuilder = ClientProtos.Get.newBuilder();
+ getBuilder.setRow(ByteString.copyFromUtf8("row"));
+ Column.Builder columnBuilder = Column.newBuilder();
+ columnBuilder.setFamily(ByteString.copyFromUtf8("f1"));
+ columnBuilder.addQualifier(ByteString.copyFromUtf8("c1"));
+ columnBuilder.addQualifier(ByteString.copyFromUtf8("c2"));
+ getBuilder.addColumn(columnBuilder.build());
+
+ columnBuilder.clear();
+ columnBuilder.setFamily(ByteString.copyFromUtf8("f2"));
+ getBuilder.addColumn(columnBuilder.build());
+ getBuilder.setLoadColumnFamiliesOnDemand(true);
+ ClientProtos.Get proto = getBuilder.build();
+ // default fields
+ assertEquals(1, proto.getMaxVersions());
+ assertEquals(true, proto.getCacheBlocks());
+
+ // set the default value for equal comparison
+ getBuilder = ClientProtos.Get.newBuilder(proto);
+ getBuilder.setMaxVersions(1);
+ getBuilder.setCacheBlocks(true);
+
+ Get get = ProtobufUtil.toGet(proto);
+ assertEquals(getBuilder.build(), ProtobufUtil.toGet(get));
+ }
+
+ /**
+ * Test Delete Mutate conversions.
+ *
+ * @throws IOException
+ */
+ @Test
+ public void testDelete() throws IOException {
+ MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
+ mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
+ mutateBuilder.setMutateType(MutationType.DELETE);
+ mutateBuilder.setTimestamp(111111);
+ ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
+ valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
+ QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
+ qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
+ qualifierBuilder.setDeleteType(DeleteType.DELETE_ONE_VERSION);
+ qualifierBuilder.setTimestamp(111222);
+ valueBuilder.addQualifierValue(qualifierBuilder.build());
+ qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
+ qualifierBuilder.setDeleteType(DeleteType.DELETE_MULTIPLE_VERSIONS);
+ qualifierBuilder.setTimestamp(111333);
+ valueBuilder.addQualifierValue(qualifierBuilder.build());
+ mutateBuilder.addColumnValue(valueBuilder.build());
+
+ MutationProto proto = mutateBuilder.build();
+ // default fields
+ assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
+
+ // set the default value for equal comparison
+ mutateBuilder = MutationProto.newBuilder(proto);
+ mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
+
+ Delete delete = ProtobufUtil.toDelete(proto);
+
+ // delete always have empty value,
+ // add empty value to the original mutate
+ for (ColumnValue.Builder column:
+ mutateBuilder.getColumnValueBuilderList()) {
+ for (QualifierValue.Builder qualifier:
+ column.getQualifierValueBuilderList()) {
+ qualifier.setValue(ByteString.EMPTY);
+ }
+ }
+ assertEquals(mutateBuilder.build(),
+ ProtobufUtil.toMutation(MutationType.DELETE, delete));
+ }
+
+ /**
+ * Test Put Mutate conversions.
+ *
+ * @throws IOException
+ */
+ @Test
+ public void testPut() throws IOException {
+ MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
+ mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
+ mutateBuilder.setMutateType(MutationType.PUT);
+ mutateBuilder.setTimestamp(111111);
+ ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
+ valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
+ QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
+ qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
+ qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
+ valueBuilder.addQualifierValue(qualifierBuilder.build());
+ qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
+ qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
+ qualifierBuilder.setTimestamp(222222);
+ valueBuilder.addQualifierValue(qualifierBuilder.build());
+ mutateBuilder.addColumnValue(valueBuilder.build());
+
+ MutationProto proto = mutateBuilder.build();
+ // default fields
+ assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
+
+ // set the default value for equal comparison
+ mutateBuilder = MutationProto.newBuilder(proto);
+ mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
+
+ Put put = ProtobufUtil.toPut(proto);
+
+ // put value always use the default timestamp if no
+ // value level timestamp specified,
+ // add the timestamp to the original mutate
+ long timestamp = put.getTimeStamp();
+ for (ColumnValue.Builder column:
+ mutateBuilder.getColumnValueBuilderList()) {
+ for (QualifierValue.Builder qualifier:
+ column.getQualifierValueBuilderList()) {
+ if (!qualifier.hasTimestamp()) {
+ qualifier.setTimestamp(timestamp);
+ }
+ }
+ }
+ assertEquals(mutateBuilder.build(),
+ ProtobufUtil.toMutation(MutationType.PUT, put));
+ }
+
+ /**
+ * Test basic Scan conversions.
+ *
+ * @throws IOException
+ */
+ @Test
+ public void testScan() throws IOException {
+ ClientProtos.Scan.Builder scanBuilder = ClientProtos.Scan.newBuilder();
+ scanBuilder.setStartRow(ByteString.copyFromUtf8("row1"));
+ scanBuilder.setStopRow(ByteString.copyFromUtf8("row2"));
+ Column.Builder columnBuilder = Column.newBuilder();
+ columnBuilder.setFamily(ByteString.copyFromUtf8("f1"));
+ columnBuilder.addQualifier(ByteString.copyFromUtf8("c1"));
+ columnBuilder.addQualifier(ByteString.copyFromUtf8("c2"));
+ scanBuilder.addColumn(columnBuilder.build());
+
+ columnBuilder.clear();
+ columnBuilder.setFamily(ByteString.copyFromUtf8("f2"));
+ scanBuilder.addColumn(columnBuilder.build());
+
+ ClientProtos.Scan proto = scanBuilder.build();
+
+ // Verify default values
+ assertEquals(1, proto.getMaxVersions());
+ assertEquals(true, proto.getCacheBlocks());
+
+ // Verify fields survive ClientProtos.Scan -> Scan -> ClientProtos.Scan
+ // conversion
+ scanBuilder = ClientProtos.Scan.newBuilder(proto);
+ scanBuilder.setMaxVersions(2);
+ scanBuilder.setCacheBlocks(false);
+ scanBuilder.setCaching(1024);
+ ClientProtos.Scan expectedProto = scanBuilder.build();
+
+ ClientProtos.Scan actualProto = ProtobufUtil.toScan(
+ ProtobufUtil.toScan(expectedProto));
+ assertEquals(expectedProto, actualProto);
+ }
+
+ @Test
+ public void testToCell() throws Exception {
+ KeyValue kv1 =
+ new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
+ KeyValue kv2 =
+ new KeyValue(Bytes.toBytes("bbb"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
+ KeyValue kv3 =
+ new KeyValue(Bytes.toBytes("ccc"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]);
+ byte[] arr = new byte[kv1.getLength() + kv2.getLength() + kv3.getLength()];
+ System.arraycopy(kv1.getBuffer(), kv1.getOffset(), arr, 0, kv1.getLength());
+ System.arraycopy(kv2.getBuffer(), kv2.getOffset(), arr, kv1.getLength(), kv2.getLength());
+ System.arraycopy(kv3.getBuffer(), kv3.getOffset(), arr, kv1.getLength() + kv2.getLength(),
+ kv3.getLength());
+ ByteBuffer dbb = ByteBuffer.allocateDirect(arr.length);
+ dbb.put(arr);
+ ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(dbb, kv1.getLength(), kv2.getLength());
+ CellProtos.Cell cell = ProtobufUtil.toCell(offheapKV);
+ Cell newOffheapKV = ProtobufUtil.toCell(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
+ assertTrue(CellComparator.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
+ }
+
+ /**
+ * Test Increment Mutate conversions.
+ *
+ * @throws IOException
+ */
+ @Test
+ public void testIncrement() throws IOException {
+ long timeStamp = 111111;
+ MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
+ mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
+ mutateBuilder.setMutateType(MutationProto.MutationType.INCREMENT);
+ ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
+ valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
+ QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
+ qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
+ qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(11L)));
+ qualifierBuilder.setTimestamp(timeStamp);
+ valueBuilder.addQualifierValue(qualifierBuilder.build());
+ qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
+ qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(22L)));
+ valueBuilder.addQualifierValue(qualifierBuilder.build());
+ mutateBuilder.addColumnValue(valueBuilder.build());
+
+ MutationProto proto = mutateBuilder.build();
+ // default fields
+ assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
+
+ // set the default value for equal comparison
+ mutateBuilder = MutationProto.newBuilder(proto);
+ mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
+
+ Increment increment = ProtobufUtil.toIncrement(proto, null);
+ mutateBuilder.setTimestamp(increment.getTimeStamp());
+ assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
+ }
+
+ /**
+ * Test Append Mutate conversions.
+ *
+ * @throws IOException
+ */
+ @Test
+ public void testAppend() throws IOException {
+ long timeStamp = 111111;
+ MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
+ mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
+ mutateBuilder.setMutateType(MutationType.APPEND);
+ mutateBuilder.setTimestamp(timeStamp);
+ ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
+ valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
+ QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
+ qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
+ qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
+ qualifierBuilder.setTimestamp(timeStamp);
+ valueBuilder.addQualifierValue(qualifierBuilder.build());
+ qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
+ qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
+ valueBuilder.addQualifierValue(qualifierBuilder.build());
+ mutateBuilder.addColumnValue(valueBuilder.build());
+
+ MutationProto proto = mutateBuilder.build();
+ // default fields
+ assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
+
+ // set the default value for equal comparison
+ mutateBuilder = MutationProto.newBuilder(proto);
+ mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
+
+ Append append = ProtobufUtil.toAppend(proto, null);
+
+ // append always use the latest timestamp,
+ // reset the timestamp to the original mutate
+ mutateBuilder.setTimestamp(append.getTimeStamp());
+ assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
+ }
+
+ private static ProcedureProtos.Procedure.Builder createProcedureBuilder(long procId) {
+ ProcedureProtos.Procedure.Builder builder = ProcedureProtos.Procedure.newBuilder();
+ builder.setProcId(procId);
+ builder.setClassName("java.lang.Object");
+ builder.setSubmittedTime(0);
+ builder.setState(ProcedureProtos.ProcedureState.RUNNABLE);
+ builder.setLastUpdate(0);
+
+ return builder;
+ }
+
+ private static ProcedureProtos.Procedure createProcedure(long procId) {
+ return createProcedureBuilder(procId).build();
+ }
+
+ private static LockServiceProtos.LockedResource createLockedResource(
+ LockServiceProtos.LockedResourceType resourceType, String resourceName,
+ LockServiceProtos.LockType lockType,
+ ProcedureProtos.Procedure exclusiveLockOwnerProcedure, int sharedLockCount) {
+ LockServiceProtos.LockedResource.Builder build = LockServiceProtos.LockedResource.newBuilder();
+ build.setResourceType(resourceType);
+ build.setResourceName(resourceName);
+ build.setLockType(lockType);
+ if (exclusiveLockOwnerProcedure != null) {
+ build.setExclusiveLockOwnerProcedure(exclusiveLockOwnerProcedure);
+ }
+ build.setSharedLockCount(sharedLockCount);
+
+ return build.build();
+ }
+
+ @Test
+ public void testProcedureInfo() {
+ ProcedureProtos.Procedure.Builder builder = createProcedureBuilder(1);
+ ByteString stateBytes = ByteString.copyFrom(new byte[] { 65 });
+ BytesValue state = BytesValue.newBuilder().setValue(stateBytes).build();
+ builder.addStateMessage(Any.pack(state));
+ ProcedureProtos.Procedure procedure = builder.build();
+
+ String procJson = ProtobufUtil.toProcedureJson(Lists.newArrayList(procedure));
+ assertEquals("[{"
+ + "\"className\":\"java.lang.Object\","
+ + "\"procId\":\"1\","
+ + "\"submittedTime\":\"0\","
+ + "\"state\":\"RUNNABLE\","
+ + "\"lastUpdate\":\"0\","
+ + "\"stateMessage\":[{\"value\":\"QQ==\"}]"
+ + "}]", procJson);
+ }
+
+ @Test
+ public void testServerLockInfo() {
+ LockServiceProtos.LockedResource resource = createLockedResource(
+ LockServiceProtos.LockedResourceType.SERVER, "server",
+ LockServiceProtos.LockType.SHARED, null, 2);
+
+ String lockJson = ProtobufUtil.toLockJson(Lists.newArrayList(resource));
+ assertEquals("[{"
+ + "\"resourceType\":\"SERVER\","
+ + "\"resourceName\":\"server\","
+ + "\"lockType\":\"SHARED\","
+ + "\"sharedLockCount\":2"
+ + "}]", lockJson);
+ }
+
+ @Test
+ public void testNamespaceLockInfo() {
+ LockServiceProtos.LockedResource resource = createLockedResource(
+ LockServiceProtos.LockedResourceType.NAMESPACE, "ns",
+ LockServiceProtos.LockType.EXCLUSIVE, createProcedure(2), 0);
+
+ String lockJson = ProtobufUtil.toLockJson(Lists.newArrayList(resource));
+ assertEquals("[{"
+ + "\"resourceType\":\"NAMESPACE\","
+ + "\"resourceName\":\"ns\","
+ + "\"lockType\":\"EXCLUSIVE\","
+ + "\"exclusiveLockOwnerProcedure\":{"
+ + "\"className\":\"java.lang.Object\","
+ + "\"procId\":\"2\","
+ + "\"submittedTime\":\"0\","
+ + "\"state\":\"RUNNABLE\","
+ + "\"lastUpdate\":\"0\""
+ + "},"
+ + "\"sharedLockCount\":0"
+ + "}]", lockJson);
+ }
+
+ @Test
+ public void testTableLockInfo() {
+ LockServiceProtos.LockedResource resource = createLockedResource(
+ LockServiceProtos.LockedResourceType.TABLE, "table",
+ LockServiceProtos.LockType.SHARED, null, 2);
+
+ String lockJson = ProtobufUtil.toLockJson(Lists.newArrayList(resource));
+ assertEquals("[{"
+ + "\"resourceType\":\"TABLE\","
+ + "\"resourceName\":\"table\","
+ + "\"lockType\":\"SHARED\","
+ + "\"sharedLockCount\":2"
+ + "}]", lockJson);
+ }
+
+ @Test
+ public void testRegionLockInfo() {
+ LockServiceProtos.LockedResource resource = createLockedResource(
+ LockServiceProtos.LockedResourceType.REGION, "region",
+ LockServiceProtos.LockType.EXCLUSIVE, createProcedure(3), 0);
+
+ String lockJson = ProtobufUtil.toLockJson(Lists.newArrayList(resource));
+ assertEquals("[{"
+ + "\"resourceType\":\"REGION\","
+ + "\"resourceName\":\"region\","
+ + "\"lockType\":\"EXCLUSIVE\","
+ + "\"exclusiveLockOwnerProcedure\":{"
+ + "\"className\":\"java.lang.Object\","
+ + "\"procId\":\"3\","
+ + "\"submittedTime\":\"0\","
+ + "\"state\":\"RUNNABLE\","
+ + "\"lastUpdate\":\"0\""
+ + "},"
+ + "\"sharedLockCount\":0"
+ + "}]", lockJson);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
deleted file mode 100644
index 36dabdd..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
+++ /dev/null
@@ -1,215 +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.hbase;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.NonceKey;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * Procedure information
- */
-@InterfaceAudience.Public
-public class ProcedureInfo implements Cloneable {
- private final long procId;
- private final String procName;
- private final String procOwner;
- private final ProcedureState procState;
- private final long parentId;
- private final NonceKey nonceKey;
- private final IOException exception;
- private final long lastUpdate;
- private final long submittedTime;
- private final byte[] result;
-
- private long clientAckTime = -1;
-
- @InterfaceAudience.Private
- public ProcedureInfo(
- final long procId,
- final String procName,
- final String procOwner,
- final ProcedureState procState,
- final long parentId,
- final NonceKey nonceKey,
- final IOException exception,
- final long lastUpdate,
- final long submittedTime,
- final byte[] result) {
- this.procId = procId;
- this.procName = procName;
- this.procOwner = procOwner;
- this.procState = procState;
- this.parentId = parentId;
- this.nonceKey = nonceKey;
- this.lastUpdate = lastUpdate;
- this.submittedTime = submittedTime;
-
- // If the procedure is completed, we should treat exception and result differently
- this.exception = exception;
- this.result = result;
- }
-
- @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="CN_IDIOM_NO_SUPER_CALL",
- justification="Intentional; calling super class clone doesn't make sense here.")
- public ProcedureInfo clone() {
- return new ProcedureInfo(procId, procName, procOwner, procState, parentId, nonceKey,
- exception, lastUpdate, submittedTime, result);
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append(procName);
- sb.append(" pid=");
- sb.append(procId);
- if (hasParentId()) {
- sb.append(", ppid=");
- sb.append(parentId);
- }
- if (hasOwner()) {
- sb.append(", owner=");
- sb.append(procOwner);
- }
- sb.append(", state=");
- sb.append(procState);
-
- long now = EnvironmentEdgeManager.currentTime();
- sb.append(", submittedTime=");
- sb.append(StringUtils.formatTime(now - submittedTime));
- sb.append(" ago, lastUpdate=");
- sb.append(StringUtils.formatTime(now - submittedTime));
- sb.append(" ago");
-
- if (isFailed()) {
- sb.append(", exception=\"");
- sb.append(this.exception.getMessage());
- sb.append("\"");
- }
- return sb.toString();
- }
-
- public long getProcId() {
- return procId;
- }
-
- public String getProcName() {
- return procName;
- }
-
- public boolean hasOwner() {
- return procOwner != null;
- }
-
- public String getProcOwner() {
- return procOwner;
- }
-
- public ProcedureState getProcState() {
- return procState;
- }
-
- public boolean hasParentId() {
- return (parentId != -1);
- }
-
- public long getParentId() {
- return parentId;
- }
-
- public NonceKey getNonceKey() {
- return nonceKey;
- }
-
- public boolean isFailed() {
- return exception != null;
- }
-
- public IOException getException() {
- if (isFailed()) {
- return this.exception;
- }
- return null;
- }
-
- public String getExceptionFullMessage() {
- assert isFailed();
- final IOException e = getException();
- return e.getCause() + " - " + e.getMessage();
- }
-
- public boolean hasResultData() {
- return result != null;
- }
-
- public byte[] getResult() {
- return result;
- }
-
- public long getSubmittedTime() {
- return submittedTime;
- }
-
- public long getLastUpdate() {
- return lastUpdate;
- }
-
- public long executionTime() {
- return lastUpdate - submittedTime;
- }
-
- @InterfaceAudience.Private
- public boolean hasClientAckTime() {
- return clientAckTime != -1;
- }
-
- @InterfaceAudience.Private
- public long getClientAckTime() {
- return clientAckTime;
- }
-
- @InterfaceAudience.Private
- public void setClientAckTime(final long timestamp) {
- this.clientAckTime = timestamp;
- }
-
- /**
- * Check if the user is this procedure's owner
- * @param procInfo the procedure to check
- * @param user the user
- * @return true if the user is the owner of the procedure,
- * false otherwise or the owner is unknown.
- */
- @InterfaceAudience.Private
- public static boolean isProcedureOwner(final ProcedureInfo procInfo, final User user) {
- if (user == null) {
- return false;
- }
- String procOwner = procInfo.getProcOwner();
- if (procOwner == null) {
- return false;
- }
- return procOwner.equals(user.getShortName());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-common/src/main/java/org/apache/hadoop/hbase/procedure2/LockInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/procedure2/LockInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/procedure2/LockInfo.java
deleted file mode 100644
index 30ecee8..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/procedure2/LockInfo.java
+++ /dev/null
@@ -1,128 +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.hbase.procedure2;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@InterfaceAudience.Public
-public class LockInfo {
- @InterfaceAudience.Public
- public enum ResourceType {
- SERVER, NAMESPACE, TABLE, REGION
- }
-
- @InterfaceAudience.Public
- public enum LockType {
- EXCLUSIVE, SHARED
- }
-
- @InterfaceAudience.Public
- public static class WaitingProcedure {
- private LockType lockType;
- private ProcedureInfo procedure;
-
- public WaitingProcedure() {
- }
-
- public LockType getLockType() {
- return lockType;
- }
-
- public void setLockType(LockType lockType) {
- this.lockType = lockType;
- }
-
- public ProcedureInfo getProcedure() {
- return procedure;
- }
-
- public void setProcedure(ProcedureInfo procedure) {
- this.procedure = procedure;
- }
- }
-
- private ResourceType resourceType;
- private String resourceName;
- private LockType lockType;
- private ProcedureInfo exclusiveLockOwnerProcedure;
- private int sharedLockCount;
- private final List<WaitingProcedure> waitingProcedures;
-
- public LockInfo() {
- waitingProcedures = new ArrayList<>();
- }
-
- public ResourceType getResourceType() {
- return resourceType;
- }
-
- public void setResourceType(ResourceType resourceType) {
- this.resourceType = resourceType;
- }
-
- public String getResourceName() {
- return resourceName;
- }
-
- public void setResourceName(String resourceName) {
- this.resourceName = resourceName;
- }
-
- public LockType getLockType() {
- return lockType;
- }
-
- public void setLockType(LockType lockType) {
- this.lockType = lockType;
- }
-
- public ProcedureInfo getExclusiveLockOwnerProcedure() {
- return exclusiveLockOwnerProcedure;
- }
-
- public void setExclusiveLockOwnerProcedure(
- ProcedureInfo exclusiveLockOwnerProcedure) {
- this.exclusiveLockOwnerProcedure = exclusiveLockOwnerProcedure;
- }
-
- public int getSharedLockCount() {
- return sharedLockCount;
- }
-
- public void setSharedLockCount(int sharedLockCount) {
- this.sharedLockCount = sharedLockCount;
- }
-
- public List<WaitingProcedure> getWaitingProcedures() {
- return waitingProcedures;
- }
-
- public void setWaitingProcedures(List<WaitingProcedure> waitingProcedures) {
- this.waitingProcedures.clear();
- this.waitingProcedures.addAll(waitingProcedures);
- }
-
- public void addWaitingProcedure(WaitingProcedure waitingProcedure) {
- waitingProcedures.add(waitingProcedure);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JRubyFormat.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JRubyFormat.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JRubyFormat.java
new file mode 100644
index 0000000..8d85b9d
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JRubyFormat.java
@@ -0,0 +1,151 @@
+/*
+ * 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.hbase.util;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.com.google.common.escape.Escaper;
+import org.apache.hadoop.hbase.shaded.com.google.common.escape.Escapers;
+
+/**
+ * Utility class for converting objects to JRuby.
+ *
+ * It handles null, Boolean, Number, String, byte[], List<Object>, Map<String, Object> structures.
+ *
+ * <p>
+ * E.g.
+ * <pre>
+ * Map<String, Object> map = new LinkedHashMap<>();
+ * map.put("null", null);
+ * map.put("boolean", true);
+ * map.put("number", 1);
+ * map.put("string", "str");
+ * map.put("binary", new byte[] { 1, 2, 3 });
+ * map.put("list", Lists.newArrayList(1, "2", true));
+ * </pre>
+ * </p>
+ *
+ * <p>
+ * Calling {@link #print(Object)} method will result:
+ * <pre>
+ * { null => '', boolean => 'true', number => '1', string => 'str', binary => '010203', list => [ '1', '2', 'true' ] }
+ * </pre>
+ * </p>
+ */
+@InterfaceAudience.Private
+public final class JRubyFormat {
+ private static final Escaper escaper;
+
+ static {
+ escaper = Escapers.builder()
+ .addEscape('\\', "\\\\")
+ .addEscape('\'', "\\'")
+ .addEscape('\n', "\\n")
+ .addEscape('\r', "\\r")
+ .addEscape('\t', "\\t")
+ .addEscape('\f', "\\f")
+ .build();
+ }
+
+ private JRubyFormat() {
+ }
+
+ private static String escape(Object object) {
+ if (object == null) {
+ return "";
+ } else {
+ return escaper.escape(object.toString());
+ }
+ }
+
+ @SuppressWarnings({ "unchecked" })
+ private static void appendJRuby(StringBuilder builder, Object object) {
+ if (object == null) {
+ builder.append("''");
+ } else if (object instanceof List) {
+ builder.append("[");
+
+ boolean first = true;
+
+ for (Object element: (List<Object>)object) {
+ if (first) {
+ first = false;
+ builder.append(" ");
+ } else {
+ builder.append(", ");
+ }
+
+ appendJRuby(builder, element);
+ }
+
+ if (!first) {
+ builder.append(" ");
+ }
+
+ builder.append("]");
+ } else if (object instanceof Map) {
+ builder.append("{");
+
+ boolean first = true;
+
+ for (Entry<String, Object> entry: ((Map<String, Object>)object).entrySet()) {
+ if (first) {
+ first = false;
+ builder.append(" ");
+ } else {
+ builder.append(", ");
+ }
+
+ String key = entry.getKey();
+ String escapedKey = escape(key);
+
+ if (key.equals(escapedKey)) {
+ builder.append(key);
+ } else {
+ builder.append("'").append(escapedKey).append("'");
+ }
+
+ builder.append(" => ");
+ appendJRuby(builder, entry.getValue());
+ }
+
+ if (!first) {
+ builder.append(" ");
+ }
+
+ builder.append("}");
+ } else if (object instanceof byte[]) {
+ String byteString = Bytes.toHex((byte[])object);
+ builder.append("'").append(escape(byteString)).append("'");
+ } else {
+ builder.append("'").append(escape(object)).append("'");
+ }
+ }
+
+ public static String print(Object object) {
+ StringBuilder builder = new StringBuilder();
+
+ appendJRuby(builder, object);
+
+ return builder.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestJRubyFormat.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestJRubyFormat.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestJRubyFormat.java
new file mode 100644
index 0000000..96b3da0
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestJRubyFormat.java
@@ -0,0 +1,54 @@
+/*
+ * 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.hbase.util;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestJRubyFormat {
+ @Test
+ public void testPrint() {
+ Map<String, Object> map = new LinkedHashMap<>();
+ map.put("null", null);
+ map.put("boolean", true);
+ map.put("number", 1);
+ map.put("string", "str");
+ map.put("binary", new byte[] { 1, 2, 3 });
+ map.put("list", Lists.newArrayList(1, "2", true));
+
+ String jrubyString = JRubyFormat.print(map);
+ assertEquals("{ null => '', boolean => 'true', number => '1', "
+ + "string => 'str', binary => '010203', "
+ + "list => [ '1', '2', 'true' ] }", jrubyString);
+ }
+
+ @Test
+ public void testEscape() {
+ String jrubyString = JRubyFormat.print("\\\'\n\r\t\f");
+ assertEquals("'\\\\\\'\\n\\r\\t\\f'", jrubyString);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockType.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockType.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockType.java
new file mode 100644
index 0000000..e4d867d
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockType.java
@@ -0,0 +1,26 @@
+/*
+ * 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.hbase.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public enum LockType {
+ EXCLUSIVE, SHARED
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResource.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResource.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResource.java
new file mode 100644
index 0000000..e3320ab
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResource.java
@@ -0,0 +1,69 @@
+/*
+ * 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.hbase.procedure2;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.LockedResourceType;
+
+@InterfaceAudience.Private
+public class LockedResource {
+ private final LockedResourceType resourceType;
+ private final String resourceName;
+ private final LockType lockType;
+ private final Procedure<?> exclusiveLockOwnerProcedure;
+ private final int sharedLockCount;
+ private final List<Procedure<?>> waitingProcedures;
+
+ public LockedResource(LockedResourceType resourceType, String resourceName,
+ LockType lockType, Procedure<?> exclusiveLockOwnerProcedure,
+ int sharedLockCount, List<Procedure<?>> waitingProcedures) {
+ this.resourceType = resourceType;
+ this.resourceName = resourceName;
+ this.lockType = lockType;
+ this.exclusiveLockOwnerProcedure = exclusiveLockOwnerProcedure;
+ this.sharedLockCount = sharedLockCount;
+ this.waitingProcedures = waitingProcedures;
+ }
+
+ public LockedResourceType getResourceType() {
+ return resourceType;
+ }
+
+ public String getResourceName() {
+ return resourceName;
+ }
+
+ public LockType getLockType() {
+ return lockType;
+ }
+
+ public Procedure<?> getExclusiveLockOwnerProcedure() {
+ return exclusiveLockOwnerProcedure;
+ }
+
+ public int getSharedLockCount() {
+ return sharedLockCount;
+ }
+
+ public List<Procedure<?>> getWaitingProcedures() {
+ return waitingProcedures;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
new file mode 100644
index 0000000..29820f1
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
@@ -0,0 +1,26 @@
+/*
+ * 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.hbase.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public enum LockedResourceType {
+ SERVER, NAMESPACE, TABLE, REGION
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 335e83c..db488c9 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
@@ -165,17 +163,17 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
* The user-level code of the procedure may have some state to
* persist (e.g. input arguments or current position in the processing state) to
* be able to resume on failure.
- * @param stream the stream that will contain the user serialized data
+ * @param serializer stores the serializable state
*/
- protected abstract void serializeStateData(final OutputStream stream)
+ protected abstract void serializeStateData(final ProcedureStateSerializer serializer)
throws IOException;
/**
* Called on store load to allow the user to decode the previously serialized
* state.
- * @param stream the stream that contains the user serialized data
+ * @param serializer contains the serialized state
*/
- protected abstract void deserializeStateData(final InputStream stream)
+ protected abstract void deserializeStateData(final ProcedureStateSerializer serializer)
throws IOException;
/**
@@ -184,7 +182,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
* Framework will call this method just before it invokes {@link #execute(Object)}.
* It calls {@link #releaseLock(Object)} after the call to execute.
*
- * <p>If you need to hold the lock for the life of the Procdure -- i.e. you do not
+ * <p>If you need to hold the lock for the life of the Procedure -- i.e. you do not
* want any other Procedure interfering while this Procedure is running, see
* {@link #holdLock(Object)}.
*
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index d0052f6..9337530 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -22,8 +22,6 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
@@ -241,7 +239,7 @@ public class ProcedureExecutor<TEnvironment> {
}
/**
- * Map the the procId returned by submitProcedure(), the Root-ProcID, to the ProcedureInfo.
+ * Map the the procId returned by submitProcedure(), the Root-ProcID, to the Procedure.
* Once a Root-Procedure completes (success or failure), the result will be added to this map.
* The user of ProcedureExecutor should call getResult(procId) to get the result.
*/
@@ -750,14 +748,22 @@ public class ProcedureExecutor<TEnvironment> {
}
}
- private static class FailedProcedure<TEnvironment> extends Procedure<TEnvironment> {
+ public static class FailedProcedure<TEnvironment> extends Procedure<TEnvironment> {
private String procName;
- public FailedProcedure(NonceKey nonceKey, String procName, User owner,
- IOException exception) {
+ public FailedProcedure() {
+ }
+
+ public FailedProcedure(long procId, String procName, User owner,
+ NonceKey nonceKey, IOException exception) {
this.procName = procName;
- setNonceKey(nonceKey);
+ setProcId(procId);
+ setState(ProcedureState.ROLLEDBACK);
setOwner(owner);
+ setNonceKey(nonceKey);
+ long currentTime = EnvironmentEdgeManager.currentTime();
+ setSubmittedTime(currentTime);
+ setLastUpdate(currentTime);
setFailure(Objects.toString(exception.getMessage(), ""), exception);
}
@@ -785,11 +791,13 @@ public class ProcedureExecutor<TEnvironment> {
}
@Override
- protected void serializeStateData(OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
@Override
- protected void deserializeStateData(InputStream stream) throws IOException {
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
}
@@ -809,7 +817,9 @@ public class ProcedureExecutor<TEnvironment> {
final Long procId = nonceKeysToProcIdsMap.get(nonceKey);
if (procId == null || completed.containsKey(procId)) return;
- Procedure proc = new FailedProcedure(nonceKey, procName, procOwner, exception);
+ Procedure<?> proc = new FailedProcedure(procId.longValue(),
+ procName, procOwner, nonceKey, exception);
+
completed.putIfAbsent(procId, new CompletedProcedureRetainer(proc));
}
@@ -1045,15 +1055,17 @@ public class ProcedureExecutor<TEnvironment> {
}
/**
- * List procedures.
+ * Get procedures.
* @return the procedures in a list
*/
- public List<Procedure> listProcedures() {
- final List<Procedure> procedureLists = new ArrayList<>(procedures.size() + completed.size());
- procedureLists.addAll(procedures.values());
+ public List<Procedure<?>> getProcedures() {
+ final List<Procedure<?>> procedureLists = new ArrayList<>(procedures.size() + completed.size());
+ for (Procedure<?> procedure : procedures.values()) {
+ procedureLists.add(procedure);
+ }
// Note: The procedure could show up twice in the list with different state, as
// it could complete after we walk through procedures list and insert into
- // procedureList - it is ok, as we will use the information in the ProcedureInfo
+ // procedureList - it is ok, as we will use the information in the Procedure
// to figure it out; to prevent this would increase the complexity of the logic.
for (CompletedProcedureRetainer retainer: completed.values()) {
procedureLists.add(retainer.getProcedure());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java
index b148dae..596ff21 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java
@@ -18,9 +18,7 @@
package org.apache.hadoop.hbase.procedure2;
-import java.io.InputStream;
-import java.io.OutputStream;
-
+import java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -58,12 +56,12 @@ public abstract class ProcedureInMemoryChore<TEnvironment> extends Procedure<TEn
}
@Override
- public void serializeStateData(final OutputStream stream) {
- throw new UnsupportedOperationException();
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
@Override
- public void deserializeStateData(final InputStream stream) {
- throw new UnsupportedOperationException();
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
index a5a126d..1e4240a 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
@@ -125,13 +125,13 @@ public interface ProcedureScheduler {
* List lock queues.
* @return the locks
*/
- // TODO: This seems to be the wrong place to hang this method.
- List<LockInfo> listLocks();
+ List<LockedResource> getLocks();
/**
- * @return {@link LockInfo} for resource of specified type & name. null if resource is not locked.
+ * @return {@link LockedResource} for resource of specified type & name. null if resource is not locked.
*/
- LockInfo getLockInfoForResource(LockInfo.ResourceType resourceType, String resourceName);
+ LockedResource getLockResource(LockedResourceType resourceType, String resourceName);
+
/**
* Returns the number of elements in this queue.
* @return the number of elements in this queue.
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureStateSerializer.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureStateSerializer.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureStateSerializer.java
new file mode 100644
index 0000000..03842d9
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureStateSerializer.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.hbase.procedure2;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
+
+public interface ProcedureStateSerializer {
+ void serialize(Message message) throws IOException;
+
+ <M extends Message> M deserialize(Class<M> clazz) throws IOException;
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
index 3232f2b..2381abd 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
@@ -18,21 +18,22 @@
package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
+import java.io.InputStream;
import java.lang.reflect.Constructor;
import java.lang.reflect.Modifier;
-
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.ProcedureState;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Any;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.NonceKey;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-
/**
* Helper to convert to/from ProcedureProtos
*/
@@ -85,6 +86,69 @@ public final class ProcedureUtil {
// ==========================================================================
/**
+ * A serializer for our Procedures. Instead of the previous serializer, it
+ * uses the stateMessage list to store the internal state of the Procedures.
+ */
+ private static class StateSerializer implements ProcedureStateSerializer {
+ private final ProcedureProtos.Procedure.Builder builder;
+ private int deserializeIndex;
+
+ public StateSerializer(ProcedureProtos.Procedure.Builder builder) {
+ this.builder = builder;
+ }
+
+ @Override
+ public void serialize(Message message) throws IOException {
+ Any packedMessage = Any.pack(message);
+ builder.addStateMessage(packedMessage);
+ }
+
+ @Override
+ public <M extends Message> M deserialize(Class<M> clazz)
+ throws IOException {
+ if (deserializeIndex >= builder.getStateMessageCount()) {
+ throw new IOException("Invalid state message index: " + deserializeIndex);
+ }
+
+ try {
+ Any packedMessage = builder.getStateMessage(deserializeIndex++);
+ return packedMessage.unpack(clazz);
+ } catch (InvalidProtocolBufferException e) {
+ throw e.unwrapIOException();
+ }
+ }
+ }
+
+ /**
+ * A serializer (deserializer) for those Procedures which were serialized
+ * before this patch. It deserializes the old, binary stateData field.
+ */
+ private static class CompatStateSerializer implements ProcedureStateSerializer {
+ private InputStream inputStream;
+
+ public CompatStateSerializer(InputStream inputStream) {
+ this.inputStream = inputStream;
+ }
+
+ @Override
+ public void serialize(Message message) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public <M extends Message> M deserialize(Class<M> clazz)
+ throws IOException {
+ Parser<M> parser = (Parser<M>) Internal.getDefaultInstance(clazz).getParserForType();
+ try {
+ return parser.parseDelimitedFrom(inputStream);
+ } catch (InvalidProtocolBufferException e) {
+ throw e.unwrapIOException();
+ }
+ }
+ }
+
+ /**
* Helper to convert the procedure to protobuf.
* Used by ProcedureStore implementations.
*/
@@ -130,15 +194,8 @@ public final class ProcedureUtil {
builder.setResult(UnsafeByteOperations.unsafeWrap(result));
}
- final ByteString.Output stateStream = ByteString.newOutput();
- try {
- proc.serializeStateData(stateStream);
- if (stateStream.size() > 0) {
- builder.setStateData(stateStream.toByteString());
- }
- } finally {
- stateStream.close();
- }
+ ProcedureStateSerializer serializer = new StateSerializer(builder);
+ proc.serializeStateData(serializer);
if (proc.getNonceKey() != null) {
builder.setNonceGroup(proc.getNonceKey().getNonceGroup());
@@ -198,87 +255,62 @@ public final class ProcedureUtil {
proc.setNonceKey(new NonceKey(proto.getNonceGroup(), proto.getNonce()));
}
- // we want to call deserialize even when the stream is empty, mainly for testing.
- proc.deserializeStateData(proto.getStateData().newInput());
+ ProcedureStateSerializer serializer = null;
+
+ if (proto.getStateMessageCount() > 0) {
+ serializer = new StateSerializer(proto.toBuilder());
+ } else if (proto.hasStateData()) {
+ InputStream inputStream = proto.getStateData().newInput();
+ serializer = new CompatStateSerializer(inputStream);
+ }
+
+ if (serializer != null) {
+ proc.deserializeStateData(serializer);
+ }
return proc;
}
// ==========================================================================
- // convert to and from ProcedureInfo object
+ // convert from LockedResource object
// ==========================================================================
- /**
- * @return Convert the current {@link ProcedureInfo} into a Protocol Buffers Procedure
- * instance.
- */
- public static ProcedureProtos.Procedure convertToProtoProcedure(final ProcedureInfo procInfo) {
- final ProcedureProtos.Procedure.Builder builder = ProcedureProtos.Procedure.newBuilder();
+ public static LockServiceProtos.LockedResourceType convertToProtoResourceType(
+ LockedResourceType resourceType) {
+ return LockServiceProtos.LockedResourceType.valueOf(resourceType.name());
+ }
- builder.setClassName(procInfo.getProcName());
- builder.setProcId(procInfo.getProcId());
- builder.setSubmittedTime(procInfo.getSubmittedTime());
- builder.setState(ProcedureProtos.ProcedureState.valueOf(procInfo.getProcState().name()));
- builder.setLastUpdate(procInfo.getLastUpdate());
+ public static LockServiceProtos.LockType convertToProtoLockType(LockType lockType) {
+ return LockServiceProtos.LockType.valueOf(lockType.name());
+ }
- if (procInfo.hasParentId()) {
- builder.setParentId(procInfo.getParentId());
- }
+ public static LockServiceProtos.LockedResource convertToProtoLockedResource(
+ LockedResource lockedResource) throws IOException
+ {
+ LockServiceProtos.LockedResource.Builder builder =
+ LockServiceProtos.LockedResource.newBuilder();
- if (procInfo.hasOwner()) {
- builder.setOwner(procInfo.getProcOwner());
- }
+ builder
+ .setResourceType(convertToProtoResourceType(lockedResource.getResourceType()))
+ .setResourceName(lockedResource.getResourceName())
+ .setLockType(convertToProtoLockType(lockedResource.getLockType()));
- if (procInfo.isFailed()) {
- builder.setException(ForeignExceptionUtil.toProtoForeignException(procInfo.getException()));
- }
+ Procedure<?> exclusiveLockOwnerProcedure = lockedResource.getExclusiveLockOwnerProcedure();
- if (procInfo.hasResultData()) {
- builder.setResult(UnsafeByteOperations.unsafeWrap(procInfo.getResult()));
+ if (exclusiveLockOwnerProcedure != null) {
+ ProcedureProtos.Procedure exclusiveLockOwnerProcedureProto =
+ convertToProtoProcedure(exclusiveLockOwnerProcedure);
+ builder.setExclusiveLockOwnerProcedure(exclusiveLockOwnerProcedureProto);
}
- return builder.build();
- }
+ builder.setSharedLockCount(lockedResource.getSharedLockCount());
- /**
- * Helper to convert the protobuf object.
- * @return Convert the current Protocol Buffers Procedure to {@link ProcedureInfo}
- * instance.
- */
- public static ProcedureInfo convertToProcedureInfo(final ProcedureProtos.Procedure procProto) {
- NonceKey nonceKey = null;
- if (procProto.getNonce() != HConstants.NO_NONCE) {
- nonceKey = new NonceKey(procProto.getNonceGroup(), procProto.getNonce());
+ for (Procedure<?> waitingProcedure : lockedResource.getWaitingProcedures()) {
+ ProcedureProtos.Procedure waitingProcedureProto =
+ convertToProtoProcedure(waitingProcedure);
+ builder.addWaitingProcedures(waitingProcedureProto);
}
- return new ProcedureInfo(procProto.getProcId(), procProto.getClassName(),
- procProto.hasOwner() ? procProto.getOwner() : null,
- convertToProcedureState(procProto.getState()),
- procProto.hasParentId() ? procProto.getParentId() : -1, nonceKey,
- procProto.hasException() ?
- ForeignExceptionUtil.toIOException(procProto.getException()) : null,
- procProto.getLastUpdate(), procProto.getSubmittedTime(),
- procProto.hasResult() ? procProto.getResult().toByteArray() : null);
- }
-
- public static ProcedureState convertToProcedureState(ProcedureProtos.ProcedureState state) {
- return ProcedureState.valueOf(state.name());
- }
-
- public static ProcedureInfo convertToProcedureInfo(final Procedure proc) {
- return convertToProcedureInfo(proc, null);
- }
-
- /**
- * Helper to create the ProcedureInfo from Procedure.
- */
- public static ProcedureInfo convertToProcedureInfo(final Procedure proc,
- final NonceKey nonceKey) {
- final RemoteProcedureException exception = proc.hasException() ? proc.getException() : null;
- return new ProcedureInfo(proc.getProcId(), proc.toStringClass(), proc.getOwner(),
- convertToProcedureState(proc.getState()),
- proc.hasParent() ? proc.getParentProcId() : -1, nonceKey,
- exception != null ? exception.unwrapRemoteIOException() : null,
- proc.getLastUpdate(), proc.getSubmittedTime(), proc.getResult());
+ return builder.build();
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
index 64bb278..f03653f 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
@@ -19,9 +19,6 @@
package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData;
@@ -69,15 +66,17 @@ public abstract class SequentialProcedure<TEnvironment> extends Procedure<TEnvir
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
SequentialProcedureData.Builder data = SequentialProcedureData.newBuilder();
data.setExecuted(executed);
- data.build().writeDelimitedTo(stream);
+ serializer.serialize(data.build());
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- SequentialProcedureData data = SequentialProcedureData.parseDelimitedFrom(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ SequentialProcedureData data = serializer.deserialize(SequentialProcedureData.class);
executed = data.getExecuted();
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
index 69c59c8..3c2445c 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
@@ -78,12 +78,13 @@ public class SimpleProcedureScheduler extends AbstractProcedureScheduler {
}
@Override
- public List<LockInfo> listLocks() {
+ public List<LockedResource> getLocks() {
return Collections.emptyList();
}
@Override
- public LockInfo getLockInfoForResource(LockInfo.ResourceType resourceType, String resourceName) {
+ public LockedResource getLockResource(LockedResourceType resourceType,
+ String resourceName) {
return null;
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
index 5de5066..25dfe8b 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@@ -285,17 +283,19 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
StateMachineProcedureData.Builder data = StateMachineProcedureData.newBuilder();
for (int i = 0; i < stateCount; ++i) {
data.addState(states[i]);
}
- data.build().writeDelimitedTo(stream);
+ serializer.serialize(data.build());
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- StateMachineProcedureData data = StateMachineProcedureData.parseDelimitedFrom(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ StateMachineProcedureData data = serializer.deserialize(StateMachineProcedureData.class);
stateCount = data.getStateCount();
if (stateCount > 0) {
states = new int[stateCount];
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 5cdbc35..99d3c28 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -23,8 +23,6 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Set;
import java.util.concurrent.Callable;
@@ -37,11 +35,12 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
-import org.apache.hadoop.hbase.io.util.StreamUtils;
import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.BytesValue;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.util.NonceKey;
import org.apache.hadoop.hbase.util.Threads;
@@ -367,11 +366,13 @@ public class ProcedureTestingUtility {
protected boolean abort(TEnv env) { return false; }
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
}
@@ -416,19 +417,23 @@ public class ProcedureTestingUtility {
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
- StreamUtils.writeRawVInt32(stream, data != null ? data.length : 0);
- if (data != null) stream.write(data);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ ByteString dataString = ByteString.copyFrom((data == null) ? new byte[0] : data);
+ BytesValue.Builder builder = BytesValue.newBuilder().setValue(dataString);
+ serializer.serialize(builder.build());
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- int len = StreamUtils.readRawVarint32(stream);
- if (len > 0) {
- data = new byte[len];
- stream.read(data);
- } else {
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ BytesValue bytesValue = serializer.deserialize(BytesValue.class);
+ ByteString dataString = bytesValue.getValue();
+
+ if (dataString.isEmpty()) {
data = null;
+ } else {
+ data = dataString.toByteArray();
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java
index b81e0f9..ce9795f 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureEvents.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
@@ -28,10 +26,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
-import org.apache.hadoop.hbase.io.util.StreamUtils;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.NoopProcedure;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Int32Value;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -42,8 +40,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
@Category({MasterTests.class, SmallTests.class})
public class TestProcedureEvents {
@@ -163,15 +159,23 @@ public class TestProcedureEvents {
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
- StreamUtils.writeRawVInt32(stream, ntimeouts.get());
- StreamUtils.writeRawVInt32(stream, maxTimeouts);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ Int32Value.Builder ntimeoutsBuilder = Int32Value.newBuilder().setValue(ntimeouts.get());
+ serializer.serialize(ntimeoutsBuilder.build());
+
+ Int32Value.Builder maxTimeoutsBuilder = Int32Value.newBuilder().setValue(maxTimeouts);
+ serializer.serialize(maxTimeoutsBuilder.build());
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- ntimeouts.set(StreamUtils.readRawVarint32(stream));
- maxTimeouts = StreamUtils.readRawVarint32(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ Int32Value ntimeoutsValue = serializer.deserialize(Int32Value.class);
+ ntimeouts.set(ntimeoutsValue.getValue());
+
+ Int32Value maxTimeoutsValue = serializer.deserialize(Int32Value.class);
+ maxTimeouts = maxTimeoutsValue.getValue();
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java
index 9681bfb..f1dadb9 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.CountDownLatch;
@@ -31,6 +29,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Int32Value;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -382,17 +381,19 @@ public class TestProcedureRecovery {
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
- stream.write(Bytes.toBytes(iResult));
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
+ Int32Value.Builder builder = Int32Value.newBuilder().setValue(iResult);
+ serializer.serialize(builder.build());
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
- byte[] data = new byte[4];
- stream.read(data);
- iResult = Bytes.toInt(data);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
+ Int32Value value = serializer.deserialize(Int32Value.class);
+ iResult = value.getValue();
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
index bd614e3..80264f5 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.concurrent.atomic.AtomicLong;
@@ -29,9 +27,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
-import org.apache.hadoop.hbase.io.util.StreamUtils;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Int64Value;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -195,13 +193,17 @@ public class TestProcedureReplayOrder {
protected boolean abort(TestProcedureEnv env) { return true; }
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
- StreamUtils.writeLong(stream, execId);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ Int64Value.Builder builder = Int64Value.newBuilder().setValue(execId);
+ serializer.serialize(builder.build());
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- execId = StreamUtils.readLong(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ Int64Value value = serializer.deserialize(Int64Value.class);
+ execId = value.getValue();
step = 2;
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java
index 0146bc7..f86df2d 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureSuspended.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.procedure2;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
@@ -251,11 +249,13 @@ public class TestProcedureSuspended {
protected boolean abort(TestProcEnv env) { return false; }
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
index 78daf5a..af25108 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
@@ -20,9 +20,6 @@ package org.apache.hadoop.hbase.procedure2;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -57,11 +54,13 @@ public class TestProcedureToString {
}
@Override
- protected void serializeStateData(OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
@Override
- protected void deserializeStateData(InputStream stream) throws IOException {
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java
index 7f98b80..dec5854 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureUtil.java
@@ -18,11 +18,7 @@
package org.apache.hadoop.hbase.procedure2;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.util.JsonFormat;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -34,8 +30,6 @@ import static org.junit.Assert.assertEquals;
@Category({MasterTests.class, SmallTests.class})
public class TestProcedureUtil {
- private static final Log LOG = LogFactory.getLog(TestProcedureUtil.class);
-
@Test
public void testValidation() throws Exception {
ProcedureUtil.validateClass(new TestProcedure(10));
@@ -49,34 +43,15 @@ public class TestProcedureUtil {
@Test
public void testConvert() throws Exception {
// check Procedure to protobuf conversion
- final TestProcedure proc1 = new TestProcedure(10);
+ final TestProcedure proc1 = new TestProcedure(10, 1, new byte[] { 65 });
final ProcedureProtos.Procedure proto1 = ProcedureUtil.convertToProtoProcedure(proc1);
final TestProcedure proc2 = (TestProcedure)ProcedureUtil.convertToProcedure(proto1);
final ProcedureProtos.Procedure proto2 = ProcedureUtil.convertToProtoProcedure(proc2);
assertEquals(false, proto2.hasResult());
assertEquals("Procedure protobuf does not match", proto1, proto2);
-
- // remove the state-data from the procedure protobuf to compare it to the gen ProcedureInfo
- final ProcedureProtos.Procedure pbproc = proto2.toBuilder().clearStateData().build();
-
- // check ProcedureInfo to protobuf conversion
- final ProcedureInfo protoInfo1 = ProcedureUtil.convertToProcedureInfo(proc1);
- final ProcedureProtos.Procedure proto3 = ProcedureUtil.convertToProtoProcedure(protoInfo1);
- final ProcedureInfo protoInfo2 = ProcedureUtil.convertToProcedureInfo(proto3);
- final ProcedureProtos.Procedure proto4 = ProcedureUtil.convertToProtoProcedure(protoInfo2);
- assertEquals("ProcedureInfo protobuf does not match", proto3, proto4);
- assertEquals("ProcedureInfo/Procedure protobuf does not match", pbproc, proto3);
- assertEquals("ProcedureInfo/Procedure protobuf does not match", pbproc, proto4);
}
public static class TestProcedureNoDefaultConstructor extends TestProcedure {
public TestProcedureNoDefaultConstructor(int x) {}
}
-
- public static void main(final String [] args) throws Exception {
- final TestProcedure proc1 = new TestProcedure(10);
- final ProcedureProtos.Procedure proto1 = ProcedureUtil.convertToProtoProcedure(proc1);
- JsonFormat.Printer printer = JsonFormat.printer().omittingInsignificantWhitespace();
- System.out.println(printer.print(proto1));
- }
}
[2/5] hbase git commit: HBASE-18106 Redo ProcedureInfo and LockInfo
Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
index f19195e..84fc7e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.List;
import org.apache.commons.logging.Log;
@@ -33,6 +31,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
@@ -166,8 +165,9 @@ public class AddColumnFamilyProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.AddColumnFamilyStateData.Builder addCFMsg =
MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
@@ -179,15 +179,16 @@ public class AddColumnFamilyProcedure
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
}
- addCFMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(addCFMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.AddColumnFamilyStateData addCFMsg =
- MasterProcedureProtos.AddColumnFamilyStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.AddColumnFamilyStateData.class);
setUser(MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
cfDescriptor = ProtobufUtil.toColumnFamilyDescriptor(addCFMsg.getColumnfamilySchema());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
index cc39f53..7e21ad6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@@ -47,6 +45,7 @@ import org.apache.hadoop.hbase.master.MetricsSnapshot;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsRegions;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -244,8 +243,9 @@ public class CloneSnapshotProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.CloneSnapshotStateData.Builder cloneSnapshotMsg =
MasterProcedureProtos.CloneSnapshotStateData.newBuilder()
@@ -271,15 +271,16 @@ public class CloneSnapshotProcedure
cloneSnapshotMsg.addParentToChildRegionsPairList(parentToChildrenPair);
}
}
- cloneSnapshotMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(cloneSnapshotMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.CloneSnapshotStateData cloneSnapshotMsg =
- MasterProcedureProtos.CloneSnapshotStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.CloneSnapshotStateData.class);
setUser(MasterProcedureUtil.toUserInfo(cloneSnapshotMsg.getUserInfo()));
snapshot = cloneSnapshotMsg.getSnapshot();
tableDescriptor = ProtobufUtil.toTableDescriptor(cloneSnapshotMsg.getTableSchema());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index 7d65126..2cbf8f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -19,9 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -29,6 +26,7 @@ import org.apache.hadoop.hbase.NamespaceExistException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
@@ -135,21 +133,23 @@ public class CreateNamespaceProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.CreateNamespaceStateData.Builder createNamespaceMsg =
MasterProcedureProtos.CreateNamespaceStateData.newBuilder().setNamespaceDescriptor(
ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
- createNamespaceMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(createNamespaceMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.CreateNamespaceStateData createNamespaceMsg =
- MasterProcedureProtos.CreateNamespaceStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.CreateNamespaceStateData.class);
nsDescriptor = ProtobufUtil.toNamespaceDescriptor(createNamespaceMsg.getNamespaceDescriptor());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index 14604fd..a5c1584 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.List;
@@ -39,6 +37,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -183,8 +182,9 @@ public class CreateTableProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.CreateTableStateData.Builder state =
MasterProcedureProtos.CreateTableStateData.newBuilder()
@@ -195,15 +195,16 @@ public class CreateTableProcedure
state.addRegionInfo(HRegionInfo.convert(hri));
}
}
- state.build().writeDelimitedTo(stream);
+ serializer.serialize(state.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.CreateTableStateData state =
- MasterProcedureProtos.CreateTableStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.CreateTableStateData.class);
setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
tableDescriptor = ProtobufUtil.toTableDescriptor(state.getTableSchema());
if (state.getRegionInfoCount() == 0) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
index 9ec814a..12726ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.List;
import org.apache.commons.logging.Log;
@@ -32,6 +30,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -172,8 +171,9 @@ public class DeleteColumnFamilyProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.DeleteColumnFamilyStateData.Builder deleteCFMsg =
MasterProcedureProtos.DeleteColumnFamilyStateData.newBuilder()
@@ -185,14 +185,15 @@ public class DeleteColumnFamilyProcedure
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
}
- deleteCFMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(deleteCFMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.DeleteColumnFamilyStateData deleteCFMsg =
- MasterProcedureProtos.DeleteColumnFamilyStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.DeleteColumnFamilyStateData.class);
setUser(MasterProcedureUtil.toUserInfo(deleteCFMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(deleteCFMsg.getTableName());
familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
index d91a6e1..a66e77c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -20,9 +20,6 @@ package org.apache.hadoop.hbase.master.procedure;
import java.io.FileNotFoundException;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
@@ -35,6 +32,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
@@ -147,8 +145,9 @@ public class DeleteNamespaceProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.DeleteNamespaceStateData.Builder deleteNamespaceMsg =
MasterProcedureProtos.DeleteNamespaceStateData.newBuilder().setNamespaceName(namespaceName);
@@ -156,15 +155,16 @@ public class DeleteNamespaceProcedure
deleteNamespaceMsg.setNamespaceDescriptor(
ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
}
- deleteNamespaceMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(deleteNamespaceMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.DeleteNamespaceStateData deleteNamespaceMsg =
- MasterProcedureProtos.DeleteNamespaceStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.DeleteNamespaceStateData.class);
namespaceName = deleteNamespaceMsg.getNamespaceName();
if (deleteNamespaceMsg.hasNamespaceDescriptor()) {
nsDescriptor =
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index 04dfc60..0f665f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.List;
@@ -48,6 +46,7 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -203,8 +202,9 @@ public class DeleteTableProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.DeleteTableStateData.Builder state =
MasterProcedureProtos.DeleteTableStateData.newBuilder()
@@ -215,15 +215,16 @@ public class DeleteTableProcedure
state.addRegionInfo(HRegionInfo.convert(hri));
}
}
- state.build().writeDelimitedTo(stream);
+ serializer.serialize(state.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.DeleteTableStateData state =
- MasterProcedureProtos.DeleteTableStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.DeleteTableStateData.class);
setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
tableName = ProtobufUtil.toTableName(state.getTableName());
if (state.getRegionInfoCount() == 0) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
index 0872a01..20eca94 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
@@ -19,9 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -33,6 +30,7 @@ import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState;
@@ -171,8 +169,9 @@ public class DisableTableProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.DisableTableStateData.Builder disableTableMsg =
MasterProcedureProtos.DisableTableStateData.newBuilder()
@@ -180,15 +179,16 @@ public class DisableTableProcedure
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setSkipTableStateCheck(skipTableStateCheck);
- disableTableMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(disableTableMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.DisableTableStateData disableTableMsg =
- MasterProcedureProtos.DisableTableStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.DisableTableStateData.class);
setUser(MasterProcedureUtil.toUserInfo(disableTableMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(disableTableMsg.getTableName());
skipTableStateCheck = disableTableMsg.getSkipTableStateCheck();
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
index 4f4b5b1..ff43d27 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
@@ -19,9 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -32,6 +29,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableState;
@@ -170,8 +168,9 @@ public class EnableTableProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.EnableTableStateData.Builder enableTableMsg =
MasterProcedureProtos.EnableTableStateData.newBuilder()
@@ -179,15 +178,16 @@ public class EnableTableProcedure
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setSkipTableStateCheck(skipTableStateCheck);
- enableTableMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(enableTableMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.EnableTableStateData enableTableMsg =
- MasterProcedureProtos.EnableTableStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.EnableTableStateData.class);
setUser(MasterProcedureUtil.toUserInfo(enableTableMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(enableTableMsg.getTableName());
skipTableStateCheck = enableTableMsg.getSkipTableStateCheck();
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 564b86d..e14c037 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.hbase.master.procedure;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -32,7 +30,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@@ -42,11 +39,13 @@ import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
import org.apache.hadoop.hbase.procedure2.AbstractProcedureScheduler;
import org.apache.hadoop.hbase.procedure2.LockAndQueue;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.procedure2.LockStatus;
+import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
+import org.apache.hadoop.hbase.procedure2.LockedResourceType;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureDeque;
-import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.AvlUtil.AvlIterableList;
import org.apache.hadoop.hbase.util.AvlUtil.AvlKeyComparator;
import org.apache.hadoop.hbase.util.AvlUtil.AvlLinkedNode;
@@ -238,57 +237,42 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
return pollResult;
}
- private LockInfo createLockInfo(LockInfo.ResourceType resourceType,
+ private LockedResource createLockedResource(LockedResourceType resourceType,
String resourceName, LockAndQueue queue) {
- LockInfo info = new LockInfo();
-
- info.setResourceType(resourceType);
- info.setResourceName(resourceName);
+ LockType lockType;
+ Procedure<?> exclusiveLockOwnerProcedure;
+ int sharedLockCount;
if (queue.hasExclusiveLock()) {
- info.setLockType(LockInfo.LockType.EXCLUSIVE);
-
- Procedure<?> exclusiveLockOwnerProcedure = queue.getExclusiveLockOwnerProcedure();
- ProcedureInfo exclusiveLockOwnerProcedureInfo =
- ProcedureUtil.convertToProcedureInfo(exclusiveLockOwnerProcedure);
- info.setExclusiveLockOwnerProcedure(exclusiveLockOwnerProcedureInfo);
- } else if (queue.getSharedLockCount() > 0) {
- info.setLockType(LockInfo.LockType.SHARED);
- info.setSharedLockCount(queue.getSharedLockCount());
+ lockType = LockType.EXCLUSIVE;
+ exclusiveLockOwnerProcedure = queue.getExclusiveLockOwnerProcedure();
+ sharedLockCount = 0;
+ } else {
+ lockType = LockType.SHARED;
+ exclusiveLockOwnerProcedure = null;
+ sharedLockCount = queue.getSharedLockCount();
}
+ List<Procedure<?>> waitingProcedures = new ArrayList<>();
+
for (Procedure<?> procedure : queue) {
if (!(procedure instanceof LockProcedure)) {
continue;
}
- LockProcedure lockProcedure = (LockProcedure)procedure;
- LockInfo.WaitingProcedure waitingProcedure = new LockInfo.WaitingProcedure();
-
- switch (lockProcedure.getType()) {
- case EXCLUSIVE:
- waitingProcedure.setLockType(LockInfo.LockType.EXCLUSIVE);
- break;
- case SHARED:
- waitingProcedure.setLockType(LockInfo.LockType.SHARED);
- break;
- }
-
- ProcedureInfo procedureInfo = ProcedureUtil.convertToProcedureInfo(lockProcedure);
- waitingProcedure.setProcedure(procedureInfo);
-
- info.addWaitingProcedure(waitingProcedure);
+ waitingProcedures.add(procedure);
}
- return info;
+ return new LockedResource(resourceType, resourceName, lockType,
+ exclusiveLockOwnerProcedure, sharedLockCount, waitingProcedures);
}
@Override
- public List<LockInfo> listLocks() {
+ public List<LockedResource> getLocks() {
schedLock();
try {
- List<LockInfo> lockInfos = new ArrayList<>();
+ List<LockedResource> lockedResources = new ArrayList<>();
for (Entry<ServerName, LockAndQueue> entry : locking.serverLocks
.entrySet()) {
@@ -296,9 +280,9 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
LockAndQueue queue = entry.getValue();
if (queue.isLocked()) {
- LockInfo lockInfo = createLockInfo(LockInfo.ResourceType.SERVER,
- serverName, queue);
- lockInfos.add(lockInfo);
+ LockedResource lockedResource =
+ createLockedResource(LockedResourceType.SERVER, serverName, queue);
+ lockedResources.add(lockedResource);
}
}
@@ -308,9 +292,9 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
LockAndQueue queue = entry.getValue();
if (queue.isLocked()) {
- LockInfo lockInfo = createLockInfo(LockInfo.ResourceType.NAMESPACE,
- namespaceName, queue);
- lockInfos.add(lockInfo);
+ LockedResource lockedResource =
+ createLockedResource(LockedResourceType.NAMESPACE, namespaceName, queue);
+ lockedResources.add(lockedResource);
}
}
@@ -320,9 +304,9 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
LockAndQueue queue = entry.getValue();
if (queue.isLocked()) {
- LockInfo lockInfo = createLockInfo(LockInfo.ResourceType.TABLE,
- tableName, queue);
- lockInfos.add(lockInfo);
+ LockedResource lockedResource =
+ createLockedResource(LockedResourceType.TABLE, tableName, queue);
+ lockedResources.add(lockedResource);
}
}
@@ -331,20 +315,21 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
LockAndQueue queue = entry.getValue();
if (queue.isLocked()) {
- LockInfo lockInfo = createLockInfo(LockInfo.ResourceType.REGION,
- regionName, queue);
- lockInfos.add(lockInfo);
+ LockedResource lockedResource =
+ createLockedResource(LockedResourceType.REGION, regionName, queue);
+ lockedResources.add(lockedResource);
}
}
- return lockInfos;
+ return lockedResources;
} finally {
schedUnlock();
}
}
@Override
- public LockInfo getLockInfoForResource(LockInfo.ResourceType resourceType, String resourceName) {
+ public LockedResource getLockResource(LockedResourceType resourceType,
+ String resourceName) {
LockAndQueue queue = null;
schedLock();
try {
@@ -363,7 +348,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
break;
}
- return queue != null ? createLockInfo(resourceType, resourceName, queue) : null;
+ return queue != null ? createLockedResource(resourceType, resourceName, queue) : null;
} finally {
schedUnlock();
}
@@ -624,17 +609,17 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
/**
* Get lock info for a resource of specified type and name and log details
*/
- protected void logLockInfoForResource(LockInfo.ResourceType resourceType, String resourceName) {
+ protected void logLockedResource(LockedResourceType resourceType, String resourceName) {
if (!LOG.isDebugEnabled()) {
return;
}
- LockInfo lockInfo = getLockInfoForResource(resourceType, resourceName);
- if (lockInfo != null) {
+ LockedResource lockedResource = getLockResource(resourceType, resourceName);
+ if (lockedResource != null) {
String msg = resourceType.toString() + " '" + resourceName + "', shared lock count=" +
- lockInfo.getSharedLockCount();
+ lockedResource.getSharedLockCount();
- ProcedureInfo proc = lockInfo.getExclusiveLockOwnerProcedure();
+ Procedure<?> proc = lockedResource.getExclusiveLockOwnerProcedure();
if (proc != null) {
msg += ", exclusively locked by procId=" + proc.getProcId();
}
@@ -657,13 +642,13 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
final LockAndQueue tableLock = locking.getTableLock(table);
if (!namespaceLock.trySharedLock()) {
waitProcedure(namespaceLock, procedure);
- logLockInfoForResource(LockInfo.ResourceType.NAMESPACE, namespace);
+ logLockedResource(LockedResourceType.NAMESPACE, namespace);
return true;
}
if (!tableLock.tryExclusiveLock(procedure)) {
namespaceLock.releaseSharedLock();
waitProcedure(tableLock, procedure);
- logLockInfoForResource(LockInfo.ResourceType.TABLE, table.getNameAsString());
+ logLockedResource(LockedResourceType.TABLE, table.getNameAsString());
return true;
}
removeFromRunQueue(tableRunQueue, getTableQueue(table));
@@ -920,7 +905,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
locking.getTableLock(TableName.NAMESPACE_TABLE_NAME);
if (!systemNamespaceTableLock.trySharedLock()) {
waitProcedure(systemNamespaceTableLock, procedure);
- logLockInfoForResource(LockInfo.ResourceType.TABLE,
+ logLockedResource(LockedResourceType.TABLE,
TableName.NAMESPACE_TABLE_NAME.getNameAsString());
return true;
}
@@ -929,7 +914,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
if (!namespaceLock.tryExclusiveLock(procedure)) {
systemNamespaceTableLock.releaseSharedLock();
waitProcedure(namespaceLock, procedure);
- logLockInfoForResource(LockInfo.ResourceType.NAMESPACE, namespace);
+ logLockedResource(LockedResourceType.NAMESPACE, namespace);
return true;
}
return false;
@@ -982,7 +967,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
return false;
}
waitProcedure(lock, procedure);
- logLockInfoForResource(LockInfo.ResourceType.SERVER, serverName.getServerName());
+ logLockedResource(LockedResourceType.SERVER, serverName.getServerName());
return true;
} finally {
schedUnlock();
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
index ac86dab..8fc9d28 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
@@ -19,9 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
@@ -31,6 +28,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
@@ -159,8 +157,9 @@ public class ModifyColumnFamilyProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.ModifyColumnFamilyStateData.Builder modifyCFMsg =
MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
@@ -172,15 +171,16 @@ public class ModifyColumnFamilyProcedure
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedtableDescriptor));
}
- modifyCFMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(modifyCFMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.ModifyColumnFamilyStateData modifyCFMsg =
- MasterProcedureProtos.ModifyColumnFamilyStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.ModifyColumnFamilyStateData.class);
setUser(MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
cfDescriptor = ProtobufUtil.toColumnFamilyDescriptor(modifyCFMsg.getColumnfamilySchema());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
index 17e7197..99c18fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
@@ -19,15 +19,13 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceState;
@@ -130,8 +128,9 @@ public class ModifyNamespaceProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.ModifyNamespaceStateData.Builder modifyNamespaceMsg =
MasterProcedureProtos.ModifyNamespaceStateData.newBuilder().setNamespaceDescriptor(
@@ -140,15 +139,16 @@ public class ModifyNamespaceProcedure
modifyNamespaceMsg.setUnmodifiedNamespaceDescriptor(
ProtobufUtil.toProtoNamespaceDescriptor(this.oldNsDescriptor));
}
- modifyNamespaceMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(modifyNamespaceMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.ModifyNamespaceStateData modifyNamespaceMsg =
- MasterProcedureProtos.ModifyNamespaceStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.ModifyNamespaceStateData.class);
newNsDescriptor =
ProtobufUtil.toNamespaceDescriptor(modifyNamespaceMsg.getNamespaceDescriptor());
if (modifyNamespaceMsg.hasUnmodifiedNamespaceDescriptor()) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
index 9741faa..0fc08c6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@@ -42,6 +40,7 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableState;
@@ -185,8 +184,9 @@ public class ModifyTableProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.ModifyTableStateData.Builder modifyTableMsg =
MasterProcedureProtos.ModifyTableStateData.newBuilder()
@@ -199,15 +199,16 @@ public class ModifyTableProcedure
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
}
- modifyTableMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(modifyTableMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.ModifyTableStateData modifyTableMsg =
- MasterProcedureProtos.ModifyTableStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.ModifyTableStateData.class);
setUser(MasterProcedureUtil.toUserInfo(modifyTableMsg.getUserInfo()));
modifiedTableDescriptor = ProtobufUtil.toTableDescriptor(modifyTableMsg.getModifiedTableSchema());
deleteColumnFamilyInModify = modifyTableMsg.getDeleteColumnFamilyInModify();
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureDescriber.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureDescriber.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureDescriber.java
new file mode 100644
index 0000000..080d6ff
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureDescriber.java
@@ -0,0 +1,83 @@
+/*
+ * 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.hbase.master.procedure;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.util.JRubyFormat;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ProcedureDescriber {
+ private ProcedureDescriber() {
+ }
+
+ private static Object parametersToObject(Procedure<?> proc) {
+ try {
+ ProcedureProtos.Procedure protoProc = ProcedureUtil.convertToProtoProcedure(proc);
+ List<Object> parameters = protoProc.getStateMessageList().stream()
+ .map((any) -> {
+ try {
+ return ProtobufMessageConverter.toJavaObject(any);
+ } catch (InvalidProtocolBufferException e) {
+ return e.toString();
+ }
+ }).collect(Collectors.toList());
+ return parameters;
+ } catch (IOException e) {
+ return e.toString();
+ }
+ }
+
+ public static String describe(Procedure<?> proc) {
+ Map<String, Object> description = new LinkedHashMap<>();
+
+ description.put("ID", proc.getProcId());
+ description.put("PARENT_ID", proc.getParentProcId());
+ description.put("STATE", proc.getState());
+ description.put("OWNER", proc.getOwner());
+ description.put("TYPE", proc.getProcName());
+ description.put("START_TIME", new Date(proc.getSubmittedTime()));
+ description.put("LAST_UPDATE", new Date(proc.getLastUpdate()));
+
+ if (proc.isFailed()) {
+ description.put("ERRORS", proc.getException().unwrapRemoteIOException().getMessage());
+ }
+ description.put("PARAMETERS", parametersToObject(proc));
+
+ return JRubyFormat.print(description);
+ }
+
+ public static String describeParameters(Procedure<?> proc) {
+ Object object = parametersToObject(proc);
+ return JRubyFormat.print(object);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
index 72f0648..7ae81ba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
@@ -38,7 +39,6 @@ import org.apache.zookeeper.KeeperException;
import java.io.IOException;
import java.io.InputStream;
-import java.io.OutputStream;
import java.util.Set;
/**
@@ -81,7 +81,7 @@ public class RecoverMetaProcedure
/**
* This constructor is also used when deserializing from a procedure store; we'll construct one
- * of these then call {@link #deserializeStateData(InputStream)}. Do not use directly.
+ * of these then call #deserializeStateData(InputStream). Do not use directly.
*/
public RecoverMetaProcedure() {
this(null, false);
@@ -183,22 +183,24 @@ public class RecoverMetaProcedure
}
@Override
- protected void serializeStateData(OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.RecoverMetaStateData.Builder state =
MasterProcedureProtos.RecoverMetaStateData.newBuilder().setShouldSplitWal(shouldSplitWal);
if (failedMetaServer != null) {
state.setFailedMetaServer(ProtobufUtil.toServerName(failedMetaServer));
}
state.setReplicaId(replicaId);
- state.build().writeDelimitedTo(stream);
+ serializer.serialize(state.build());
}
@Override
- protected void deserializeStateData(InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.RecoverMetaStateData state =
- MasterProcedureProtos.RecoverMetaStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.RecoverMetaStateData.class);
this.shouldSplitWal = state.hasShouldSplitWal() && state.getShouldSplitWal();
this.failedMetaServer = state.hasFailedMetaServer() ?
ProtobufUtil.toServerName(state.getFailedMetaServer()) : null;
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
index 4930396..754bb1d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@@ -45,6 +43,7 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MetricsSnapshot;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -229,8 +228,9 @@ public class RestoreSnapshotProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.RestoreSnapshotStateData.Builder restoreSnapshotMsg =
MasterProcedureProtos.RestoreSnapshotStateData.newBuilder()
@@ -267,15 +267,16 @@ public class RestoreSnapshotProcedure
restoreSnapshotMsg.addParentToChildRegionsPairList (parentToChildrenPair);
}
}
- restoreSnapshotMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(restoreSnapshotMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.RestoreSnapshotStateData restoreSnapshotMsg =
- MasterProcedureProtos.RestoreSnapshotStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.RestoreSnapshotStateData.class);
setUser(MasterProcedureUtil.toUserInfo(restoreSnapshotMsg.getUserInfo()));
snapshot = restoreSnapshotMsg.getSnapshot();
modifiedTableDescriptor =
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index c7af53f..9307c45 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
@@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.RegionTransitionProcedure;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
@@ -96,7 +96,7 @@ implements ServerProcedureInterface {
/**
* Used when deserializing from a procedure store; we'll construct one of these then call
- * {@link #deserializeStateData(InputStream)}. Do not use directly.
+ * #deserializeStateData(InputStream). Do not use directly.
*/
public ServerCrashProcedure() {
super();
@@ -285,8 +285,9 @@ implements ServerProcedureInterface {
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.ServerCrashStateData.Builder state =
MasterProcedureProtos.ServerCrashStateData.newBuilder().
@@ -298,15 +299,16 @@ implements ServerProcedureInterface {
state.addRegionsOnCrashedServer(HRegionInfo.convert(hri));
}
}
- state.build().writeDelimitedTo(stream);
+ serializer.serialize(state.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.ServerCrashStateData state =
- MasterProcedureProtos.ServerCrashStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.ServerCrashStateData.class);
this.serverName = ProtobufUtil.toServerName(state.getServerName());
this.carryingMeta = state.hasCarryingMeta()? state.getCarryingMeta(): false;
// shouldSplitWAL has a default over in pb so this invocation will always work.
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
index 506c67d..8f21298 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@@ -35,6 +33,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -209,8 +208,9 @@ public class TruncateTableProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
MasterProcedureProtos.TruncateTableStateData.Builder state =
MasterProcedureProtos.TruncateTableStateData.newBuilder()
@@ -226,15 +226,16 @@ public class TruncateTableProcedure
state.addRegionInfo(HRegionInfo.convert(hri));
}
}
- state.build().writeDelimitedTo(stream);
+ serializer.serialize(state.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
MasterProcedureProtos.TruncateTableStateData state =
- MasterProcedureProtos.TruncateTableStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.TruncateTableStateData.class);
setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
if (state.hasTableSchema()) {
tableDescriptor = ProtobufUtil.toTableDescriptor(state.getTableSchema());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index 979a351..81bcc0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -27,14 +27,14 @@ import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
@@ -45,9 +45,9 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.MetricsSnapshot;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -114,7 +114,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
this.snapshotManifest = SnapshotManifest.create(conf, fs, workingDir, snapshot, monitor);
this.tableLock = master.getLockManager().createMasterLock(
- snapshotTable, LockProcedure.LockType.EXCLUSIVE,
+ snapshotTable, LockType.EXCLUSIVE,
this.getClass().getName() + ": take snapshot " + snapshot.getName());
// prepare the verify
@@ -134,6 +134,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
return htd;
}
+ @Override
public TakeSnapshotHandler prepare() throws Exception {
super.prepare();
// after this, you should ensure to release this lock in case of exceptions
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 1e63b13..2229bd2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
@@ -78,9 +77,10 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.locking.LockProcedure;
-import org.apache.hadoop.hbase.master.locking.LockProcedure.LockType;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
@@ -1208,31 +1208,32 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
}
@Override
- public void preListProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
+ public void preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
- // We are delegating the authorization check to postListProcedures as we don't have
+ // We are delegating the authorization check to postGetProcedures as we don't have
// any concrete set of procedures to work with
}
@Override
- public void postListProcedures(
+ public void postGetProcedures(
ObserverContext<MasterCoprocessorEnvironment> ctx,
- List<ProcedureInfo> procInfoList) throws IOException {
- if (procInfoList.isEmpty()) {
+ List<Procedure<?>> procList) throws IOException {
+ if (procList.isEmpty()) {
return;
}
// Retains only those which passes authorization checks, as the checks weren't done as part
- // of preListProcedures.
- Iterator<ProcedureInfo> itr = procInfoList.iterator();
+ // of preGetProcedures.
+ Iterator<Procedure<?>> itr = procList.iterator();
User user = getActiveUser(ctx);
while (itr.hasNext()) {
- ProcedureInfo procInfo = itr.next();
+ Procedure<?> proc = itr.next();
try {
- if (!ProcedureInfo.isProcedureOwner(procInfo, user)) {
+ String owner = proc.getOwner();
+ if (owner == null || !owner.equals(user.getShortName())) {
// If the user is not the procedure owner, then we should further probe whether
// he can see the procedure.
- requirePermission(user, "listProcedures", Action.ADMIN);
+ requirePermission(user, "getProcedures", Action.ADMIN);
}
} catch (AccessDeniedException e) {
itr.remove();
@@ -1241,6 +1242,13 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
}
@Override
+ public void preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)
+ throws IOException {
+ User user = getActiveUser(ctx);
+ requirePermission(user, "getLocks", Action.ADMIN);
+ }
+
+ @Override
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
ServerName srcServer, ServerName destServer) throws IOException {
requirePermission(getActiveUser(c), "move", region.getTable(), null, null, Action.ADMIN);
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
index ce62fa1..63a41cc 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
@@ -29,14 +29,16 @@
import="org.apache.hadoop.hbase.HBaseConfiguration"
import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv"
- import="org.apache.hadoop.hbase.ProcedureInfo"
- import="org.apache.hadoop.hbase.procedure2.LockInfo"
+ import="org.apache.hadoop.hbase.master.procedure.ProcedureDescriber"
+ import="org.apache.hadoop.hbase.procedure2.LockedResource"
import="org.apache.hadoop.hbase.procedure2.Procedure"
import="org.apache.hadoop.hbase.procedure2.ProcedureExecutor"
+ import="org.apache.hadoop.hbase.procedure2.ProcedureUtil"
import="org.apache.hadoop.hbase.procedure2.store.wal.ProcedureWALFile"
import="org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore"
import="org.apache.hadoop.hbase.procedure2.util.StringUtils"
-
+ import="org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos"
+ import="org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil"
%>
<%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
@@ -48,7 +50,7 @@
long millisFromLastRoll = walStore.getMillisFromLastRoll();
ArrayList<ProcedureWALFile> procedureWALFiles = walStore.getActiveLogs();
Set<ProcedureWALFile> corruptedWALFiles = walStore.getCorruptedLogs();
- List<Procedure> procedures = procExecutor.listProcedures();
+ List<Procedure<?>> procedures = procExecutor.getProcedures();
Collections.sort(procedures, new Comparator<Procedure>() {
@Override
public int compare(Procedure lhs, Procedure rhs) {
@@ -58,7 +60,7 @@
}
});
- List<LockInfo> locks = master.listLocks();
+ List<LockedResource> lockedResources = master.getLocks();
%>
<!DOCTYPE html>
<?xml version="1.0" encoding="UTF-8" ?>
@@ -118,17 +120,19 @@
<th>Start Time</th>
<th>Last Update</th>
<th>Errors</th>
+ <th>Parameters</th>
</tr>
<% for (Procedure<?> proc : procedures) { %>
<tr>
<td><%= proc.getProcId() %></td>
<td><%= proc.hasParent() ? proc.getParentProcId() : "" %></td>
<td><%= escapeXml(proc.getState().toString()) %></td>
- <td><%= escapeXml(proc.getOwner()) %></td>
+ <td><%= proc.hasOwner() ? escapeXml(proc.getOwner()) : "" %></td>
<td><%= escapeXml(proc.getProcName()) %></td>
<td><%= new Date(proc.getSubmittedTime()) %></td>
<td><%= new Date(proc.getLastUpdate()) %></td>
<td><%= escapeXml(proc.isFailed() ? proc.getException().unwrapRemoteIOException().getMessage() : "") %></td>
+ <td><%= escapeXml(ProcedureDescriber.describeParameters(proc)) %></td>
</tr>
<% } %>
</table>
@@ -244,40 +248,35 @@
<h1>Locks</h1>
</div>
</div>
- <% for (LockInfo lock : locks) { %>
- <h2><%= lock.getResourceType() %>: <%= lock.getResourceName() %></h2>
+ <% for (LockedResource lockedResource : lockedResources) { %>
+ <h2><%= lockedResource.getResourceType() %>: <%= lockedResource.getResourceName() %></h2>
<%
- switch (lock.getLockType()) {
+ switch (lockedResource.getLockType()) {
case EXCLUSIVE:
%>
<p>Lock type: EXCLUSIVE</p>
- <p>Owner procedure ID: <%= lock.getExclusiveLockOwnerProcedure().getProcId() %></p>
+ <p>Owner procedure: <%= escapeXml(ProcedureDescriber.describe(lockedResource.getExclusiveLockOwnerProcedure())) %></p>
<%
break;
case SHARED:
%>
<p>Lock type: SHARED</p>
- <p>Number of shared locks: <%= lock.getSharedLockCount() %></p>
+ <p>Number of shared locks: <%= lockedResource.getSharedLockCount() %></p>
<%
break;
}
- List<LockInfo.WaitingProcedure> waitingProcedures = lock.getWaitingProcedures();
+ List<Procedure<?>> waitingProcedures = lockedResource.getWaitingProcedures();
if (!waitingProcedures.isEmpty()) {
%>
<h3>Waiting procedures</h3>
<table class="table table-striped" width="90%" >
- <tr>
- <th>Lock type</th>
- <th>Procedure ID</th>
- </tr>
- <% for (LockInfo.WaitingProcedure waitingProcedure : waitingProcedures) { %>
- <tr>
- <td><%= waitingProcedure.getLockType() %></td>
- <td><%= waitingProcedure.getProcedure().getProcId() %></td>
- </tr>
- <% } %>
+ <% for (Procedure<?> proc : procedures) { %>
+ <tr>
+ <td><%= escapeXml(ProcedureDescriber.describe(proc)) %></td>
+ </tr>
+ <% } %>
</table>
<% } %>
<% } %>
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 7bdda80..cfc4911 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -44,8 +44,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@@ -683,9 +681,15 @@ public class TestAdmin2 {
}
@Test(timeout = 300000)
- public void testListProcedures() throws Exception {
- ProcedureInfo[] procList = admin.listProcedures();
- assertTrue(procList.length >= 0);
+ public void testGetProcedures() throws Exception {
+ String procList = admin.getProcedures();
+ assertTrue(procList.startsWith("["));
+ }
+
+ @Test(timeout = 300000)
+ public void testGetLocks() throws Exception {
+ String lockList = admin.getLocks();
+ assertTrue(lockList.startsWith("["));
}
/*
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java
index b7fac06..9ef26d3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java
@@ -18,9 +18,16 @@
package org.apache.hadoop.hbase.client;
+import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.procedure.ProcedureManagerHost;
import org.apache.hadoop.hbase.procedure.SimpleMasterProcedureManager;
@@ -35,16 +42,6 @@ import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
/**
* Class to test asynchronous procedure admin operations.
*/
@@ -98,8 +95,8 @@ public class TestAsyncProcedureAdminApi extends TestAsyncAdminBase {
@Test
public void listProcedure() throws Exception {
- List<ProcedureInfo> procList = admin.listProcedures().get();
- assertTrue(procList.size() >= 0);
+ String procList = admin.getProcedures().get();
+ assertTrue(procList.startsWith("["));
}
@Test
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 1b8b27b..49ab418 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
@@ -60,7 +59,9 @@ import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
+import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
+import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -126,10 +127,10 @@ public class TestMasterObserver {
private boolean postDisableTableCalled;
private boolean preAbortProcedureCalled;
private boolean postAbortProcedureCalled;
- private boolean preListProceduresCalled;
- private boolean postListProceduresCalled;
- private boolean preListLocksCalled;
- private boolean postListLocksCalled;
+ private boolean preGetProceduresCalled;
+ private boolean postGetProceduresCalled;
+ private boolean preGetLocksCalled;
+ private boolean postGetLocksCalled;
private boolean preMoveCalled;
private boolean postMoveCalled;
private boolean preAssignCalled;
@@ -224,8 +225,10 @@ public class TestMasterObserver {
postDisableTableCalled = false;
preAbortProcedureCalled = false;
postAbortProcedureCalled = false;
- preListProceduresCalled = false;
- postListProceduresCalled = false;
+ preGetProceduresCalled = false;
+ postGetProceduresCalled = false;
+ preGetLocksCalled = false;
+ postGetLocksCalled = false;
preMoveCalled= false;
postMoveCalled = false;
preAssignCalled = false;
@@ -710,43 +713,43 @@ public class TestMasterObserver {
}
@Override
- public void preListProcedures(
+ public void preGetProcedures(
ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
- preListProceduresCalled = true;
+ preGetProceduresCalled = true;
}
@Override
- public void postListProcedures(
+ public void postGetProcedures(
ObserverContext<MasterCoprocessorEnvironment> ctx,
- List<ProcedureInfo> procInfoList) throws IOException {
- postListProceduresCalled = true;
+ List<Procedure<?>> procInfoList) throws IOException {
+ postGetProceduresCalled = true;
}
- public boolean wasListProceduresCalled() {
- return preListProceduresCalled && postListProceduresCalled;
+ public boolean wasGetProceduresCalled() {
+ return preGetProceduresCalled && postGetProceduresCalled;
}
- public boolean wasPreListProceduresCalledOnly() {
- return preListProceduresCalled && !postListProceduresCalled;
+ public boolean wasPreGetProceduresCalledOnly() {
+ return preGetProceduresCalled && !postGetProceduresCalled;
}
@Override
- public void preListLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
- preListLocksCalled = true;
+ public void preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+ preGetLocksCalled = true;
}
@Override
- public void postListLocks(ObserverContext<MasterCoprocessorEnvironment> ctx, List<LockInfo> lockInfoList)
+ public void postGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx, List<LockedResource> lockedResources)
throws IOException {
- postListLocksCalled = true;
+ postGetLocksCalled = true;
}
- public boolean wasListLocksCalled() {
- return preListLocksCalled && postListLocksCalled;
+ public boolean wasGetLocksCalled() {
+ return preGetLocksCalled && postGetLocksCalled;
}
- public boolean wasPreListLocksCalledOnly() {
- return preListLocksCalled && !postListLocksCalled;
+ public boolean wasPreGetLocksCalledOnly() {
+ return preGetLocksCalled && !postGetLocksCalled;
}
@Override
@@ -1546,14 +1549,14 @@ public class TestMasterObserver {
@Override
public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
- TableName tableName, HRegionInfo[] regionInfos, LockProcedure.LockType type,
+ TableName tableName, HRegionInfo[] regionInfos, LockType type,
String description) throws IOException {
preRequestLockCalled = true;
}
@Override
public void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
- TableName tableName, HRegionInfo[] regionInfos, LockProcedure.LockType type,
+ TableName tableName, HRegionInfo[] regionInfos, LockType type,
String description) throws IOException {
postRequestLockCalled = true;
}
@@ -2173,7 +2176,7 @@ public class TestMasterObserver {
}
@Test (timeout=180000)
- public void testListProceduresOperation() throws Exception {
+ public void testGetProceduresOperation() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
@@ -2182,14 +2185,14 @@ public class TestMasterObserver {
CPMasterObserver.class.getName());
cp.resetStates();
- master.listProcedures();
+ master.getProcedures();
assertTrue(
- "Coprocessor should be called on list procedures request",
- cp.wasListProceduresCalled());
+ "Coprocessor should be called on get procedures request",
+ cp.wasGetProceduresCalled());
}
@Test (timeout=180000)
- public void testListLocksOperation() throws Exception {
+ public void testGetLocksOperation() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
@@ -2198,10 +2201,10 @@ public class TestMasterObserver {
CPMasterObserver.class.getName());
cp.resetStates();
- master.listLocks();
+ master.getLocks();
assertTrue(
- "Coprocessor should be called on list locks request",
- cp.wasListLocksCalled());
+ "Coprocessor should be called on get locks request",
+ cp.wasGetLocksCalled());
}
private void deleteTable(Admin admin, TableName tableName) throws Exception {
@@ -2222,7 +2225,7 @@ public class TestMasterObserver {
final TableName tableName = TableName.valueOf("testLockedTable");
long procId = master.getLockManager().remoteLocks().requestTableLock(tableName,
- LockProcedure.LockType.EXCLUSIVE, "desc", null);
+ LockType.EXCLUSIVE, "desc", null);
master.getLockManager().remoteLocks().lockHeartbeat(procId, false);
assertTrue(cp.preAndPostForQueueLockAndHeartbeatLockCalled());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 9f4ce35..e96ce6d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
@@ -41,7 +40,8 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
+import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+
import com.google.protobuf.Service;
import static org.mockito.Mockito.mock;
@@ -226,12 +227,12 @@ public class MockNoopMasterServices implements MasterServices, Server {
}
@Override
- public List<ProcedureInfo> listProcedures() throws IOException {
+ public List<Procedure<?>> getProcedures() throws IOException {
return null; //To change body of implemented methods use File | Settings | File Templates.
}
@Override
- public List<LockInfo> listLocks() throws IOException {
+ public List<LockedResource> getLocks() throws IOException {
return null;
}
[5/5] hbase git commit: HBASE-18106 Redo ProcedureInfo and LockInfo
Posted by st...@apache.org.
HBASE-18106 Redo ProcedureInfo and LockInfo
Main changes:
- ProcedureInfo and LockInfo were removed, we use JSON instead of them
- Procedure and LockedResource are their server side equivalent
- Procedure protobuf state_data became obsolate, it is only kept for
reading previously written WAL
- Procedure protobuf contains a state_message field, which stores the internal
state messages (Any type instead of bytes)
- Procedure.serializeStateData and deserializeStateData were changed slightly
- Procedures internal states are available on client side
- Procedures are displayed on web UI and in shell in the following jruby format:
{ ID => '1', PARENT_ID = '-1', PARAMETERS => [ ..extra state information.. ] }
Signed-off-by: Michael Stack <st...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/359fed7b
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/359fed7b
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/359fed7b
Branch: refs/heads/master
Commit: 359fed7b4b62e4dc7818a211735630cf0e10fc14
Parents: 87c3ae9
Author: Balazs Meszaros <ba...@cloudera.com>
Authored: Wed Sep 6 15:21:32 2017 +0200
Committer: Michael Stack <st...@apache.org>
Committed: Fri Sep 8 10:24:04 2017 -0700
----------------------------------------------------------------------
.../org/apache/hadoop/hbase/client/Admin.java | 14 +-
.../apache/hadoop/hbase/client/AsyncAdmin.java | 12 +-
.../hadoop/hbase/client/AsyncHBaseAdmin.java | 10 +-
.../hbase/client/ConnectionImplementation.java | 12 +-
.../apache/hadoop/hbase/client/HBaseAdmin.java | 48 +-
.../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 92 ++--
.../client/ShortCircuitMasterConnection.java | 139 +++++-
.../protobuf/ProtobufMessageConverter.java | 159 +++++++
.../hbase/shaded/protobuf/ProtobufUtil.java | 197 +-------
.../hbase/shaded/protobuf/TestProtobufUtil.java | 466 +++++++++++++++++++
.../org/apache/hadoop/hbase/ProcedureInfo.java | 215 ---------
.../hadoop/hbase/procedure2/LockInfo.java | 128 -----
.../apache/hadoop/hbase/util/JRubyFormat.java | 151 ++++++
.../hadoop/hbase/util/TestJRubyFormat.java | 54 +++
.../hadoop/hbase/procedure2/LockType.java | 26 ++
.../hadoop/hbase/procedure2/LockedResource.java | 69 +++
.../hbase/procedure2/LockedResourceType.java | 26 ++
.../hadoop/hbase/procedure2/Procedure.java | 12 +-
.../hbase/procedure2/ProcedureExecutor.java | 42 +-
.../procedure2/ProcedureInMemoryChore.java | 12 +-
.../hbase/procedure2/ProcedureScheduler.java | 8 +-
.../procedure2/ProcedureStateSerializer.java | 28 ++
.../hadoop/hbase/procedure2/ProcedureUtil.java | 194 ++++----
.../hbase/procedure2/SequentialProcedure.java | 13 +-
.../procedure2/SimpleProcedureScheduler.java | 5 +-
.../hbase/procedure2/StateMachineProcedure.java | 12 +-
.../procedure2/ProcedureTestingUtility.java | 33 +-
.../hbase/procedure2/TestProcedureEvents.java | 26 +-
.../hbase/procedure2/TestProcedureRecovery.java | 21 +-
.../procedure2/TestProcedureReplayOrder.java | 16 +-
.../procedure2/TestProcedureSuspended.java | 8 +-
.../hbase/procedure2/TestProcedureToString.java | 9 +-
.../hbase/procedure2/TestProcedureUtil.java | 27 +-
.../hbase/procedure2/TestYieldProcedures.java | 9 +-
.../store/wal/TestWALProcedureStore.java | 21 +-
.../src/main/protobuf/LockService.proto | 21 +-
.../src/main/protobuf/Master.proto | 18 +-
.../src/main/protobuf/Procedure.proto | 4 +-
.../hbase/rsgroup/RSGroupAdminServer.java | 11 +-
.../hbase/coprocessor/MasterObserver.java | 33 +-
.../master/ExpiredMobFileCleanerChore.java | 4 +-
.../org/apache/hadoop/hbase/master/HMaster.java | 30 +-
.../hbase/master/MasterCoprocessorHost.java | 25 +-
.../hbase/master/MasterMobCompactionThread.java | 4 +-
.../hadoop/hbase/master/MasterRpcServices.java | 156 ++++++-
.../hadoop/hbase/master/MasterServices.java | 12 +-
.../hadoop/hbase/master/MobCompactionChore.java | 4 +-
.../master/assignment/AssignProcedure.java | 14 +-
.../assignment/GCMergedRegionsProcedure.java | 18 +-
.../master/assignment/GCRegionProcedure.java | 18 +-
.../assignment/MergeTableRegionsProcedure.java | 24 +-
.../master/assignment/MoveRegionProcedure.java | 18 +-
.../assignment/SplitTableRegionProcedure.java | 19 +-
.../master/assignment/UnassignProcedure.java | 18 +-
.../hbase/master/locking/LockManager.java | 33 +-
.../hbase/master/locking/LockProcedure.java | 17 +-
.../AbstractStateMachineRegionProcedure.java | 21 +-
.../procedure/AddColumnFamilyProcedure.java | 17 +-
.../procedure/CloneSnapshotProcedure.java | 17 +-
.../procedure/CreateNamespaceProcedure.java | 18 +-
.../master/procedure/CreateTableProcedure.java | 17 +-
.../procedure/DeleteColumnFamilyProcedure.java | 17 +-
.../procedure/DeleteNamespaceProcedure.java | 18 +-
.../master/procedure/DeleteTableProcedure.java | 17 +-
.../master/procedure/DisableTableProcedure.java | 18 +-
.../master/procedure/EnableTableProcedure.java | 18 +-
.../procedure/MasterProcedureScheduler.java | 111 ++---
.../procedure/ModifyColumnFamilyProcedure.java | 18 +-
.../procedure/ModifyNamespaceProcedure.java | 18 +-
.../master/procedure/ModifyTableProcedure.java | 17 +-
.../master/procedure/ProcedureDescriber.java | 83 ++++
.../master/procedure/RecoverMetaProcedure.java | 18 +-
.../procedure/RestoreSnapshotProcedure.java | 17 +-
.../master/procedure/ServerCrashProcedure.java | 18 +-
.../procedure/TruncateTableProcedure.java | 17 +-
.../master/snapshot/TakeSnapshotHandler.java | 11 +-
.../hbase/security/access/AccessController.java | 32 +-
.../hbase-webapps/master/procedures.jsp | 43 +-
.../apache/hadoop/hbase/client/TestAdmin2.java | 14 +-
.../client/TestAsyncProcedureAdminApi.java | 25 +-
.../hbase/coprocessor/TestMasterObserver.java | 75 +--
.../hbase/master/MockNoopMasterServices.java | 9 +-
.../hbase/master/locking/TestLockManager.java | 25 +-
.../hbase/master/locking/TestLockProcedure.java | 50 +-
.../procedure/TestMasterProcedureScheduler.java | 141 +++---
.../master/procedure/TestProcedureAdmin.java | 12 +-
.../hbase/procedure/TestProcedureDescriber.java | 83 ++++
.../hadoop/hbase/protobuf/TestProtobufUtil.java | 41 --
.../security/access/TestAccessController.java | 50 +-
.../hbase/shaded/protobuf/TestProtobufUtil.java | 460 ------------------
hbase-shell/src/main/ruby/hbase/admin.rb | 4 +-
.../src/main/ruby/shell/commands/list_locks.rb | 31 +-
.../main/ruby/shell/commands/list_procedures.rb | 14 +-
.../hbase/client/TestReplicationShell.java | 2 +-
.../client/procedure/ShellTestProcedure.java | 87 ++++
.../src/test/ruby/shell/list_locks_test.rb | 298 +++++++-----
.../src/test/ruby/shell/list_procedures_test.rb | 68 +++
97 files changed, 2754 insertions(+), 2131 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index b19c107..26384c9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
@@ -46,7 +45,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaRetriever;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
@@ -1454,19 +1452,19 @@ public interface Admin extends Abortable, Closeable {
final boolean mayInterruptIfRunning) throws IOException;
/**
- * List procedures
- * @return procedure list
+ * Get procedures.
+ * @return procedure list in JSON
* @throws IOException
*/
- ProcedureInfo[] listProcedures()
+ String getProcedures()
throws IOException;
/**
- * List locks.
- * @return lock list
+ * Get locks.
+ * @return lock list in JSON
* @throws IOException if a remote or network exception occurs
*/
- LockInfo[] listLocks()
+ String getLocks()
throws IOException;
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index bdd047d..923a573 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -29,13 +29,11 @@ import java.util.regex.Pattern;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Options;
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
@@ -804,15 +802,15 @@ public interface AsyncAdmin {
/**
* List procedures
- * @return procedure list wrapped by {@link CompletableFuture}
+ * @return procedure list JSON wrapped by {@link CompletableFuture}
*/
- CompletableFuture<List<ProcedureInfo>> listProcedures();
+ CompletableFuture<String> getProcedures();
/**
- * List procedure locks.
- * @return lock list wrapped by {@link CompletableFuture}
+ * List locks.
+ * @return lock list JSON wrapped by {@link CompletableFuture}
*/
- CompletableFuture<List<LockInfo>> listProcedureLocks();
+ CompletableFuture<String> getLocks();
/**
* Mark a region server as draining to prevent additional regions from getting assigned to it.
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index df077fc..b0b6ba6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -38,7 +38,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Options;
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -47,7 +46,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -469,13 +467,13 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
}
@Override
- public CompletableFuture<List<ProcedureInfo>> listProcedures() {
- return wrap(rawAdmin.listProcedures());
+ public CompletableFuture<String> getProcedures() {
+ return wrap(rawAdmin.getProcedures());
}
@Override
- public CompletableFuture<List<LockInfo>> listProcedureLocks() {
- return wrap(rawAdmin.listProcedureLocks());
+ public CompletableFuture<String> getLocks() {
+ return wrap(rawAdmin.getLocks());
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index fcd7c22..b274371 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1294,17 +1294,17 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
- public MasterProtos.ListProceduresResponse listProcedures(
+ public MasterProtos.GetProceduresResponse getProcedures(
RpcController controller,
- MasterProtos.ListProceduresRequest request) throws ServiceException {
- return stub.listProcedures(controller, request);
+ MasterProtos.GetProceduresRequest request) throws ServiceException {
+ return stub.getProcedures(controller, request);
}
@Override
- public MasterProtos.ListLocksResponse listLocks(
+ public MasterProtos.GetLocksResponse getLocks(
RpcController controller,
- MasterProtos.ListLocksRequest request) throws ServiceException {
- return stub.listLocks(controller, request);
+ MasterProtos.GetLocksRequest request) throws ServiceException {
+ return stub.getLocks(controller, request);
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index c699676..5866439 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
@@ -79,7 +78,6 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaRetriever;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
@@ -108,7 +106,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
@@ -134,9 +131,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProced
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
@@ -149,10 +150,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedur
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
@@ -180,7 +178,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMaster
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
@@ -2216,40 +2213,27 @@ public class HBaseAdmin implements Admin {
}
@Override
- public ProcedureInfo[] listProcedures() throws IOException {
- return executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection(),
+ public String getProcedures() throws IOException {
+ return executeCallable(new MasterCallable<String>(getConnection(),
getRpcControllerFactory()) {
@Override
- protected ProcedureInfo[] rpcCall() throws Exception {
- List<ProcedureProtos.Procedure> procList = master.listProcedures(
- getRpcController(), ListProceduresRequest.newBuilder().build()).getProcedureList();
- ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
- for (int i = 0; i < procList.size(); i++) {
- procInfoList[i] = ProtobufUtil.toProcedureInfo(procList.get(i));
- }
- return procInfoList;
+ protected String rpcCall() throws Exception {
+ GetProceduresRequest request = GetProceduresRequest.newBuilder().build();
+ GetProceduresResponse response = master.getProcedures(getRpcController(), request);
+ return ProtobufUtil.toProcedureJson(response.getProcedureList());
}
});
}
@Override
- public LockInfo[] listLocks() throws IOException {
- return executeCallable(new MasterCallable<LockInfo[]>(getConnection(),
+ public String getLocks() throws IOException {
+ return executeCallable(new MasterCallable<String>(getConnection(),
getRpcControllerFactory()) {
@Override
- protected LockInfo[] rpcCall() throws Exception {
- ListLocksRequest request = ListLocksRequest.newBuilder().build();
- ListLocksResponse response = master.listLocks(getRpcController(), request);
- List<LockServiceProtos.LockInfo> locksProto = response.getLockList();
-
- LockInfo[] locks = new LockInfo[locksProto.size()];
-
- for (int i = 0; i < locks.length; i++) {
- LockServiceProtos.LockInfo lockProto = locksProto.get(i);
- locks[i] = ProtobufUtil.toLockInfo(lockProto);
- }
-
- return locks;
+ protected String rpcCall() throws Exception {
+ GetLocksRequest request = GetLocksRequest.newBuilder().build();
+ GetLocksResponse response = master.getLocks(getRpcController(), request);
+ return ProtobufUtil.toLockJson(response.getLockList());
}
});
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index c5eb168..47ca32f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -37,38 +37,29 @@ import java.util.function.BiConsumer;
import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcChannel;
-
-import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
-import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
-
import java.util.stream.Stream;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Options;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
-import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.hadoop.hbase.ClusterStatus.Options;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
@@ -80,7 +71,6 @@ import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
@@ -88,13 +78,13 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
+import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
@@ -112,8 +102,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerR
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
@@ -124,40 +114,44 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceReq
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
@@ -176,12 +170,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrM
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
@@ -253,6 +243,10 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcChannel;
+
/**
* The implementation of AsyncAdmin.
* @since 2.0.0
@@ -1403,6 +1397,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return future;
}
+ @Override
public CompletableFuture<Void> addReplicationPeer(String peerId,
ReplicationPeerConfig peerConfig) {
return this
@@ -1448,6 +1443,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
.call();
}
+ @Override
public CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId) {
return this
.<ReplicationPeerConfig> newMasterCaller()
@@ -1976,27 +1972,26 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
}
@Override
- public CompletableFuture<List<ProcedureInfo>> listProcedures() {
+ public CompletableFuture<String> getProcedures() {
return this
- .<List<ProcedureInfo>> newMasterCaller()
+ .<String> newMasterCaller()
.action(
(controller, stub) -> this
- .<ListProceduresRequest, ListProceduresResponse, List<ProcedureInfo>> call(
- controller, stub, ListProceduresRequest.newBuilder().build(),
- (s, c, req, done) -> s.listProcedures(c, req, done),
- resp -> resp.getProcedureList().stream().map(ProtobufUtil::toProcedureInfo)
- .collect(Collectors.toList()))).call();
+ .<GetProceduresRequest, GetProceduresResponse, String> call(
+ controller, stub, GetProceduresRequest.newBuilder().build(),
+ (s, c, req, done) -> s.getProcedures(c, req, done),
+ resp -> ProtobufUtil.toProcedureJson(resp.getProcedureList()))).call();
}
@Override
- public CompletableFuture<List<LockInfo>> listProcedureLocks() {
+ public CompletableFuture<String> getLocks() {
return this
- .<List<LockInfo>> newMasterCaller()
+ .<String> newMasterCaller()
.action(
- (controller, stub) -> this.<ListLocksRequest, ListLocksResponse, List<LockInfo>> call(
- controller, stub, ListLocksRequest.newBuilder().build(),
- (s, c, req, done) -> s.listLocks(c, req, done), resp -> resp.getLockList().stream()
- .map(ProtobufUtil::toLockInfo).collect(Collectors.toList()))).call();
+ (controller, stub) -> this.<GetLocksRequest, GetLocksResponse, String> call(
+ controller, stub, GetLocksRequest.newBuilder().build(),
+ (s, c, req, done) -> s.getLocks(c, req, done),
+ resp -> ProtobufUtil.toLockJson(resp.getLockList()))).call();
}
@Override
@@ -2220,6 +2215,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "CREATE";
}
@@ -2231,6 +2227,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "DELETE";
}
@@ -2248,6 +2245,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "TRUNCATE";
}
@@ -2259,6 +2257,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "ENABLE";
}
@@ -2270,6 +2269,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "DISABLE";
}
@@ -2281,6 +2281,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "ADD_COLUMN_FAMILY";
}
@@ -2292,6 +2293,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "DELETE_COLUMN_FAMILY";
}
@@ -2303,6 +2305,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "MODIFY_COLUMN_FAMILY";
}
@@ -2314,6 +2317,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, namespaceName);
}
+ @Override
String getOperationType() {
return "CREATE_NAMESPACE";
}
@@ -2325,6 +2329,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, namespaceName);
}
+ @Override
String getOperationType() {
return "DELETE_NAMESPACE";
}
@@ -2336,6 +2341,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, namespaceName);
}
+ @Override
String getOperationType() {
return "MODIFY_NAMESPACE";
}
@@ -2347,6 +2353,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "MERGE_REGIONS";
}
@@ -2358,6 +2365,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
super(admin, tableName);
}
+ @Override
String getOperationType() {
return "SPLIT_REGION";
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index a8050d4..17b21cf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -22,7 +22,132 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
@@ -189,15 +314,15 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
}
@Override
- public ListProceduresResponse listProcedures(RpcController controller,
- ListProceduresRequest request) throws ServiceException {
- return stub.listProcedures(controller, request);
+ public GetProceduresResponse getProcedures(RpcController controller,
+ GetProceduresRequest request) throws ServiceException {
+ return stub.getProcedures(controller, request);
}
@Override
- public ListLocksResponse listLocks(RpcController controller,
- ListLocksRequest request) throws ServiceException {
- return stub.listLocks(controller, request);
+ public GetLocksResponse getLocks(RpcController controller,
+ GetLocksRequest request) throws ServiceException {
+ return stub.getLocks(controller, request);
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMessageConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMessageConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMessageConverter.java
new file mode 100644
index 0000000..4ad78f5
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMessageConverter.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.protobuf;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.com.google.gson.JsonArray;
+import org.apache.hadoop.hbase.shaded.com.google.gson.JsonElement;
+import org.apache.hadoop.hbase.shaded.com.google.gson.JsonObject;
+import org.apache.hadoop.hbase.shaded.com.google.gson.JsonParser;
+import org.apache.hadoop.hbase.shaded.com.google.gson.JsonPrimitive;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.BytesValue;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.util.JsonFormat;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.util.JsonFormat.TypeRegistry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * This class converts PB Messages to various representations, like:
+ * <ul>
+ * <li>JSON string: {@link #toJsonElement(MessageOrBuilder)}</li>
+ * <li>JSON object (gson): {@link #toJsonElement(MessageOrBuilder)}</li>
+ * <li>Java objects (Boolean, Number, String, List, Map):
+ * {@link #toJavaObject(JsonElement)}</li>
+ * </ul>
+ */
+@InterfaceAudience.Private
+public class ProtobufMessageConverter {
+ private static final String TYPE_KEY = "@type";
+
+ private static final JsonFormat.Printer jsonPrinter;
+
+ static {
+ TypeRegistry.Builder builder = TypeRegistry.newBuilder();
+ builder
+ .add(BytesValue.getDescriptor())
+ .add(LockServiceProtos.getDescriptor().getMessageTypes())
+ .add(MasterProcedureProtos.getDescriptor().getMessageTypes())
+ .add(ProcedureProtos.getDescriptor().getMessageTypes());
+ TypeRegistry typeRegistry = builder.build();
+ jsonPrinter = JsonFormat.printer()
+ .usingTypeRegistry(typeRegistry)
+ .omittingInsignificantWhitespace();
+ }
+
+ private ProtobufMessageConverter() {
+ }
+
+ public static String toJsonString(MessageOrBuilder messageOrBuilder)
+ throws InvalidProtocolBufferException {
+ return jsonPrinter.print(messageOrBuilder);
+ }
+
+ private static void removeTypeFromJson(JsonElement json) {
+ if (json.isJsonArray()) {
+ for (JsonElement child: json.getAsJsonArray()) {
+ removeTypeFromJson(child);
+ }
+ } else if (json.isJsonObject()) {
+ Iterator<Entry<String, JsonElement>> iterator =
+ json.getAsJsonObject().entrySet().iterator();
+
+ while (iterator.hasNext()) {
+ Entry<String, JsonElement> entry = iterator.next();
+ if (TYPE_KEY.equals(entry.getKey())) {
+ iterator.remove();
+ } else {
+ removeTypeFromJson(entry.getValue());
+ }
+ }
+ }
+ }
+
+ public static JsonElement toJsonElement(MessageOrBuilder messageOrBuilder)
+ throws InvalidProtocolBufferException {
+ return toJsonElement(messageOrBuilder, true);
+ }
+
+ public static JsonElement toJsonElement(MessageOrBuilder messageOrBuilder,
+ boolean removeType) throws InvalidProtocolBufferException {
+ String jsonString = toJsonString(messageOrBuilder);
+ JsonParser parser = new JsonParser();
+ JsonElement element = parser.parse(jsonString);
+ if (removeType) {
+ removeTypeFromJson(element);
+ }
+ return element;
+ }
+
+ private static Object toJavaObject(JsonElement element) {
+ if (element.isJsonNull()) {
+ return null;
+ } else if (element.isJsonPrimitive()) {
+ JsonPrimitive primitive = element.getAsJsonPrimitive();
+ if (primitive.isBoolean()) {
+ return primitive.getAsBoolean();
+ } else if (primitive.isNumber()) {
+ return primitive.getAsNumber();
+ } else if (primitive.isString()) {
+ return primitive.getAsString();
+ } else {
+ return null;
+ }
+ } else if (element.isJsonArray()) {
+ JsonArray array = element.getAsJsonArray();
+ List<Object> list = new ArrayList<>();
+
+ for (JsonElement arrayElement : array) {
+ Object javaObject = toJavaObject(arrayElement);
+ list.add(javaObject);
+ }
+
+ return list;
+ } else if (element.isJsonObject()) {
+ JsonObject object = element.getAsJsonObject();
+ Map<String, Object> map = new LinkedHashMap<>();
+
+ for (Entry<String, JsonElement> entry: object.entrySet()) {
+ Object javaObject = toJavaObject(entry.getValue());
+ map.put(entry.getKey(), javaObject);
+ }
+
+ return map;
+ } else {
+ return null;
+ }
+ }
+
+ public static Object toJavaObject(MessageOrBuilder messageOrBuilder)
+ throws InvalidProtocolBufferException {
+ JsonElement element = toJsonElement(messageOrBuilder);
+ return toJavaObject(element);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index e97b78d..71cd674 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -57,8 +57,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.ProcedureState;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@@ -94,8 +92,8 @@ import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.LimitInputStream;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
+import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter;
import org.apache.hadoop.hbase.quotas.QuotaScope;
import org.apache.hadoop.hbase.quotas.QuotaType;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
@@ -104,6 +102,8 @@ import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.security.visibility.CellVisibility;
+import org.apache.hadoop.hbase.shaded.com.google.gson.JsonArray;
+import org.apache.hadoop.hbase.shaded.com.google.gson.JsonElement;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
@@ -164,7 +164,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDe
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
@@ -182,9 +181,7 @@ import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DynamicClassLoader;
import org.apache.hadoop.hbase.util.ExceptionUtil;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Methods;
-import org.apache.hadoop.hbase.util.NonceKey;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.ipc.RemoteException;
@@ -202,7 +199,6 @@ import org.apache.hadoop.ipc.RemoteException;
value="DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED", justification="None. Address sometime.")
@InterfaceAudience.Private // TODO: some clients (Hive, etc) use this class
public final class ProtobufUtil {
-
private ProtobufUtil() {
}
@@ -3240,175 +3236,32 @@ public final class ProtobufUtil {
}
/**
- * @return Convert the current {@link ProcedureInfo} into a Protocol Buffers Procedure
- * instance.
- */
- public static ProcedureProtos.Procedure toProtoProcedure(ProcedureInfo procedure) {
- ProcedureProtos.Procedure.Builder builder = ProcedureProtos.Procedure.newBuilder();
-
- builder.setClassName(procedure.getProcName());
- builder.setProcId(procedure.getProcId());
- builder.setSubmittedTime(procedure.getSubmittedTime());
- builder.setState(ProcedureProtos.ProcedureState.valueOf(procedure.getProcState().name()));
- builder.setLastUpdate(procedure.getLastUpdate());
-
- if (procedure.hasParentId()) {
- builder.setParentId(procedure.getParentId());
- }
-
- if (procedure.hasOwner()) {
- builder.setOwner(procedure.getProcOwner());
- }
-
- if (procedure.isFailed()) {
- builder.setException(ForeignExceptionUtil.toProtoForeignException(procedure.getException()));
- }
-
- if (procedure.hasResultData()) {
- builder.setResult(UnsafeByteOperations.unsafeWrap(procedure.getResult()));
- }
-
- return builder.build();
- }
-
- /**
- * Helper to convert the protobuf object.
- * @return Convert the current Protocol Buffers Procedure to {@link ProcedureInfo}
- * instance.
+ * Helper to convert the protobuf Procedure to JSON String
+ * @return Convert the current Protocol Buffers Procedure to JSON String
*/
- public static ProcedureInfo toProcedureInfo(ProcedureProtos.Procedure procedureProto) {
- NonceKey nonceKey = null;
-
- if (procedureProto.getNonce() != HConstants.NO_NONCE) {
- nonceKey = new NonceKey(procedureProto.getNonceGroup(), procedureProto.getNonce());
- }
-
- return new ProcedureInfo(procedureProto.getProcId(), procedureProto.getClassName(),
- procedureProto.hasOwner() ? procedureProto.getOwner() : null,
- ProcedureState.valueOf(procedureProto.getState().name()),
- procedureProto.hasParentId() ? procedureProto.getParentId() : -1, nonceKey,
- procedureProto.hasException() ?
- ForeignExceptionUtil.toIOException(procedureProto.getException()) : null,
- procedureProto.getLastUpdate(), procedureProto.getSubmittedTime(),
- procedureProto.hasResult() ? procedureProto.getResult().toByteArray() : null);
- }
-
- public static LockServiceProtos.ResourceType toProtoResourceType(
- LockInfo.ResourceType resourceType) {
- switch (resourceType) {
- case SERVER:
- return LockServiceProtos.ResourceType.RESOURCE_TYPE_SERVER;
- case NAMESPACE:
- return LockServiceProtos.ResourceType.RESOURCE_TYPE_NAMESPACE;
- case TABLE:
- return LockServiceProtos.ResourceType.RESOURCE_TYPE_TABLE;
- case REGION:
- return LockServiceProtos.ResourceType.RESOURCE_TYPE_REGION;
- default:
- throw new IllegalArgumentException("Unknown resource type: " + resourceType);
- }
- }
-
- public static LockInfo.ResourceType toResourceType(
- LockServiceProtos.ResourceType resourceTypeProto) {
- switch (resourceTypeProto) {
- case RESOURCE_TYPE_SERVER:
- return LockInfo.ResourceType.SERVER;
- case RESOURCE_TYPE_NAMESPACE:
- return LockInfo.ResourceType.NAMESPACE;
- case RESOURCE_TYPE_TABLE:
- return LockInfo.ResourceType.TABLE;
- case RESOURCE_TYPE_REGION:
- return LockInfo.ResourceType.REGION;
- default:
- throw new IllegalArgumentException("Unknown resource type: " + resourceTypeProto);
- }
- }
-
- public static LockServiceProtos.LockType toProtoLockType(
- LockInfo.LockType lockType) {
- return LockServiceProtos.LockType.valueOf(lockType.name());
- }
-
- public static LockInfo.LockType toLockType(
- LockServiceProtos.LockType lockTypeProto) {
- return LockInfo.LockType.valueOf(lockTypeProto.name());
- }
-
- public static LockServiceProtos.WaitingProcedure toProtoWaitingProcedure(
- LockInfo.WaitingProcedure waitingProcedure) {
- LockServiceProtos.WaitingProcedure.Builder builder = LockServiceProtos.WaitingProcedure.newBuilder();
-
- ProcedureProtos.Procedure procedureProto =
- toProtoProcedure(waitingProcedure.getProcedure());
-
- builder
- .setLockType(toProtoLockType(waitingProcedure.getLockType()))
- .setProcedure(procedureProto);
-
- return builder.build();
- }
-
- public static LockInfo.WaitingProcedure toWaitingProcedure(
- LockServiceProtos.WaitingProcedure waitingProcedureProto) {
- LockInfo.WaitingProcedure waiting = new LockInfo.WaitingProcedure();
-
- waiting.setLockType(toLockType(waitingProcedureProto.getLockType()));
-
- ProcedureInfo procedure =
- toProcedureInfo(waitingProcedureProto.getProcedure());
- waiting.setProcedure(procedure);
-
- return waiting;
- }
-
- public static LockServiceProtos.LockInfo toProtoLockInfo(LockInfo lock)
- {
- LockServiceProtos.LockInfo.Builder builder = LockServiceProtos.LockInfo.newBuilder();
-
- builder
- .setResourceType(toProtoResourceType(lock.getResourceType()))
- .setResourceName(lock.getResourceName())
- .setLockType(toProtoLockType(lock.getLockType()));
-
- ProcedureInfo exclusiveLockOwnerProcedure = lock.getExclusiveLockOwnerProcedure();
-
- if (exclusiveLockOwnerProcedure != null) {
- Procedure exclusiveLockOwnerProcedureProto =
- toProtoProcedure(lock.getExclusiveLockOwnerProcedure());
- builder.setExclusiveLockOwnerProcedure(exclusiveLockOwnerProcedureProto);
- }
-
- builder.setSharedLockCount(lock.getSharedLockCount());
-
- for (LockInfo.WaitingProcedure waitingProcedure : lock.getWaitingProcedures()) {
- builder.addWaitingProcedures(toProtoWaitingProcedure(waitingProcedure));
+ public static String toProcedureJson(List<ProcedureProtos.Procedure> procProtos) {
+ JsonArray procJsons = new JsonArray(procProtos.size());
+ for (ProcedureProtos.Procedure procProto : procProtos) {
+ try {
+ JsonElement procJson = ProtobufMessageConverter.toJsonElement(procProto);
+ procJsons.add(procJson);
+ } catch (InvalidProtocolBufferException e) {
+ procJsons.add(e.toString());
+ }
}
-
- return builder.build();
+ return procJsons.toString();
}
- public static LockInfo toLockInfo(LockServiceProtos.LockInfo lockProto)
- {
- LockInfo lock = new LockInfo();
-
- lock.setResourceType(toResourceType(lockProto.getResourceType()));
- lock.setResourceName(lockProto.getResourceName());
- lock.setLockType(toLockType(lockProto.getLockType()));
-
- if (lockProto.hasExclusiveLockOwnerProcedure()) {
- ProcedureInfo exclusiveLockOwnerProcedureProto =
- toProcedureInfo(lockProto.getExclusiveLockOwnerProcedure());
-
- lock.setExclusiveLockOwnerProcedure(exclusiveLockOwnerProcedureProto);
- }
-
- lock.setSharedLockCount(lockProto.getSharedLockCount());
-
- for (LockServiceProtos.WaitingProcedure waitingProcedureProto : lockProto.getWaitingProceduresList()) {
- lock.addWaitingProcedure(toWaitingProcedure(waitingProcedureProto));
+ public static String toLockJson(List<LockServiceProtos.LockedResource> lockedResourceProtos) {
+ JsonArray lockedResourceJsons = new JsonArray(lockedResourceProtos.size());
+ for (LockServiceProtos.LockedResource lockedResourceProto : lockedResourceProtos) {
+ try {
+ JsonElement lockedResourceJson = ProtobufMessageConverter.toJsonElement(lockedResourceProto);
+ lockedResourceJsons.add(lockedResourceJson);
+ } catch (InvalidProtocolBufferException e) {
+ lockedResourceJsons.add(e.toString());
+ }
}
-
- return lock;
+ return lockedResourceJsons.toString();
}
}
[3/5] hbase git commit: HBASE-18106 Redo ProcedureInfo and LockInfo
Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java
index b1d0669..4882168 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestYieldProcedures.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.procedure2;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
@@ -336,11 +334,13 @@ public class TestYieldProcedures {
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
}
}
@@ -353,6 +353,7 @@ public class TestYieldProcedures {
public TestScheduler() {}
+ @Override
public void addFront(final Procedure proc) {
addFrontCalls++;
super.addFront(proc);
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
index 9b8c46f..44c8e12 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.LoadCounter;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure;
@@ -43,9 +44,9 @@ import org.apache.hadoop.hbase.procedure2.SequentialProcedure;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Int64Value;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.IOUtils;
import org.junit.After;
@@ -514,7 +515,7 @@ public class TestWALProcedureStore {
storeRestart(loader);
assertTrue(procStore.getCorruptedLogs() != null);
assertEquals(1, procStore.getCorruptedLogs().size());
- assertEquals(85, loader.getLoadedCount());
+ assertEquals(87, loader.getLoadedCount());
assertEquals(0, loader.getCorruptedCount());
}
@@ -911,22 +912,22 @@ public class TestWALProcedureStore {
protected boolean abort(Void env) { return false; }
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
long procId = getProcId();
if (procId % 2 == 0) {
- stream.write(Bytes.toBytes(procId));
+ Int64Value.Builder builder = Int64Value.newBuilder().setValue(procId);
+ serializer.serialize(builder.build());
}
}
@Override
- protected void deserializeStateData(InputStream stream) throws IOException {
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
long procId = getProcId();
if (procId % 2 == 0) {
- byte[] bProcId = new byte[8];
- assertEquals(8, stream.read(bProcId));
- assertEquals(procId, Bytes.toLong(bProcId));
- } else {
- assertEquals(0, stream.available());
+ Int64Value value = serializer.deserialize(Int64Value.class);
+ assertEquals(procId, value.getValue());
}
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/LockService.proto b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
index 1898e68..567dee7 100644
--- a/hbase-protocol-shaded/src/main/protobuf/LockService.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
@@ -71,25 +71,20 @@ message LockProcedureData {
optional bool is_master_lock = 6 [default = false];
}
-enum ResourceType {
- RESOURCE_TYPE_SERVER = 1;
- RESOURCE_TYPE_NAMESPACE = 2;
- RESOURCE_TYPE_TABLE = 3;
- RESOURCE_TYPE_REGION = 4;
+enum LockedResourceType {
+ SERVER = 1;
+ NAMESPACE = 2;
+ TABLE = 3;
+ REGION = 4;
}
-message WaitingProcedure {
- required LockType lock_type = 1;
- required Procedure procedure = 2;
-}
-
-message LockInfo {
- required ResourceType resource_type = 1;
+message LockedResource {
+ required LockedResourceType resource_type = 1;
optional string resource_name = 2;
required LockType lock_type = 3;
optional Procedure exclusive_lock_owner_procedure = 4;
optional int32 shared_lock_count = 5;
- repeated WaitingProcedure waitingProcedures = 6;
+ repeated Procedure waitingProcedures = 6;
}
service LockService {
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 33f9bf3..2c1694e 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -544,18 +544,18 @@ message AbortProcedureResponse {
required bool is_procedure_aborted = 1;
}
-message ListProceduresRequest {
+message GetProceduresRequest {
}
-message ListProceduresResponse {
+message GetProceduresResponse {
repeated Procedure procedure = 1;
}
-message ListLocksRequest {
+message GetLocksRequest {
}
-message ListLocksResponse {
- repeated LockInfo lock = 1;
+message GetLocksResponse {
+ repeated LockedResource lock = 1;
}
message SetQuotaRequest {
@@ -917,11 +917,11 @@ service MasterService {
returns(AbortProcedureResponse);
/** returns a list of procedures */
- rpc ListProcedures(ListProceduresRequest)
- returns(ListProceduresResponse);
+ rpc GetProcedures(GetProceduresRequest)
+ returns(GetProceduresResponse);
- rpc ListLocks(ListLocksRequest)
- returns(ListLocksResponse);
+ rpc GetLocks(GetLocksRequest)
+ returns(GetLocksResponse);
/** Add a replication peer */
rpc AddReplicationPeer(AddReplicationPeerRequest)
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
index 1a3ecf5..c13a37e 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
@@ -23,6 +23,7 @@ option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
+import "google/protobuf/any.proto";
import "ErrorHandling.proto";
enum ProcedureState {
@@ -55,7 +56,8 @@ message Procedure {
// user state/results
optional ForeignExceptionMessage exception = 10;
optional bytes result = 11; // opaque (user) result structure
- optional bytes state_data = 12; // opaque (user) procedure internal-state
+ optional bytes state_data = 12; // opaque (user) procedure internal-state - OBSOLATE
+ repeated google.protobuf.Any state_message = 15; // opaque (user) procedure internal-state
// Nonce to prevent same procedure submit by multiple times
optional uint64 nonce_group = 13 [default = 0];
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
index e3f5be5..f0cefe4 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
@@ -38,17 +38,16 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.net.Address;
-
+import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@@ -253,7 +252,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
for (TableName table: tables) {
LOG.info("Unassigning region(s) from " + table + " for table move to " + targetGroupName);
LockManager.MasterLock lock = master.getLockManager().createMasterLock(table,
- LockProcedure.LockType.EXCLUSIVE, this.getClass().getName() + ": RSGroup: table move");
+ LockType.EXCLUSIVE, this.getClass().getName() + ": RSGroup: table move");
try {
try {
lock.acquire();
@@ -420,7 +419,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
}
for (TableName table: tables) {
LockManager.MasterLock lock = master.getLockManager().createMasterLock(table,
- LockProcedure.LockType.EXCLUSIVE, this.getClass().getName() + ": RSGroup: table move");
+ LockType.EXCLUSIVE, this.getClass().getName() + ": RSGroup: table move");
try {
try {
lock.acquire();
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 8e368ba..f1cf49d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -43,7 +42,9 @@ import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
+import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
+import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
@@ -970,38 +971,38 @@ public interface MasterObserver extends Coprocessor {
throws IOException {}
/**
- * Called before a listProcedures request has been processed.
+ * Called before a getProcedures request has been processed.
* @param ctx the environment to interact with the framework and master
*/
- default void preListProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
+ default void preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {}
/**
- * Called after a listProcedures request has been processed.
+ * Called after a getProcedures request has been processed.
* @param ctx the environment to interact with the framework and master
- * @param procInfoList the list of procedures about to be returned
+ * @param procList the list of procedures about to be returned
*/
- default void postListProcedures(
+ default void postGetProcedures(
ObserverContext<MasterCoprocessorEnvironment> ctx,
- List<ProcedureInfo> procInfoList) throws IOException {}
+ List<Procedure<?>> procList) throws IOException {}
/**
- * Called before a listLocks request has been processed.
+ * Called before a getLocks request has been processed.
* @param ctx the environment to interact with the framework and master
* @throws IOException if something went wrong
*/
- default void preListLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)
+ default void preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {}
/**
- * Called after a listLocks request has been processed.
+ * Called after a getLocks request has been processed.
* @param ctx the environment to interact with the framework and master
- * @param lockInfoList the list of locks about to be returned
+ * @param lockedResources the list of locks about to be returned
* @throws IOException if something went wrong
*/
- default void postListLocks(
+ default void postGetLocks(
ObserverContext<MasterCoprocessorEnvironment> ctx,
- List<LockInfo> lockInfoList) throws IOException {}
+ List<LockedResource> lockedResources) throws IOException {}
/**
* Called prior to moving a given region from one region server to another.
@@ -1890,7 +1891,7 @@ public interface MasterObserver extends Coprocessor {
* @param ctx the environment to interact with the framework and master
*/
default void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
- TableName tableName, HRegionInfo[] regionInfos, LockProcedure.LockType type,
+ TableName tableName, HRegionInfo[] regionInfos, LockType type,
String description) throws IOException {}
/**
@@ -1898,7 +1899,7 @@ public interface MasterObserver extends Coprocessor {
* @param ctx the environment to interact with the framework and master
*/
default void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
- TableName tableName, HRegionInfo[] regionInfos, LockProcedure.LockType type,
+ TableName tableName, HRegionInfo[] regionInfos, LockType type,
String description) throws IOException {}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
index c4438bb..c4c1495 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
@@ -29,10 +29,10 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.procedure2.LockType;
/**
* The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
@@ -68,7 +68,7 @@ public class ExpiredMobFileCleanerChore extends ScheduledChore {
// clean only for mob-enabled column.
// obtain a read table lock before cleaning, synchronize with MobFileCompactionChore.
final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
- MobUtils.getTableLockName(htd.getTableName()), LockProcedure.LockType.SHARED,
+ MobUtils.getTableLockName(htd.getTableName()), LockType.SHARED,
this.getClass().getSimpleName() + ": Cleaning expired mob files");
try {
lock.acquire();
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index ad304ae..0c79c58 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.PleaseHoldException;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
@@ -138,11 +137,10 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
@@ -3051,41 +3049,35 @@ public class HMaster extends HRegionServer implements MasterServices {
}
@Override
- public List<ProcedureInfo> listProcedures() throws IOException {
+ public List<Procedure<?>> getProcedures() throws IOException {
if (cpHost != null) {
- cpHost.preListProcedures();
+ cpHost.preGetProcedures();
}
- final List<Procedure> procList = this.procedureExecutor.listProcedures();
- final List<ProcedureInfo> procInfoList = new ArrayList<>(procList.size());
-
- for (Procedure proc : procList) {
- ProcedureInfo procInfo = ProcedureUtil.convertToProcedureInfo(proc);
- procInfoList.add(procInfo);
- }
+ final List<Procedure<?>> procList = this.procedureExecutor.getProcedures();
if (cpHost != null) {
- cpHost.postListProcedures(procInfoList);
+ cpHost.postGetProcedures(procList);
}
- return procInfoList;
+ return procList;
}
@Override
- public List<LockInfo> listLocks() throws IOException {
+ public List<LockedResource> getLocks() throws IOException {
if (cpHost != null) {
- cpHost.preListLocks();
+ cpHost.preGetLocks();
}
MasterProcedureScheduler procedureScheduler = procedureExecutor.getEnvironment().getProcedureScheduler();
- final List<LockInfo> lockInfoList = procedureScheduler.listLocks();
+ final List<LockedResource> lockedResources = procedureScheduler.getLocks();
if (cpHost != null) {
- cpHost.postListLocks(lockInfoList);
+ cpHost.postGetLocks(lockedResources);
}
- return lockInfoList;
+ return lockedResources;
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 6c43fc0..eaa4f5f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -54,7 +53,9 @@ import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
+import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
+import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.security.User;
@@ -691,42 +692,42 @@ public class MasterCoprocessorHost
});
}
- public boolean preListProcedures() throws IOException {
+ public boolean preGetProcedures() throws IOException {
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
- oserver.preListProcedures(ctx);
+ oserver.preGetProcedures(ctx);
}
});
}
- public void postListProcedures(final List<ProcedureInfo> procInfoList) throws IOException {
+ public void postGetProcedures(final List<Procedure<?>> procInfoList) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
- oserver.postListProcedures(ctx, procInfoList);
+ oserver.postGetProcedures(ctx, procInfoList);
}
});
}
- public boolean preListLocks() throws IOException {
+ public boolean preGetLocks() throws IOException {
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
- oserver.preListLocks(ctx);
+ oserver.preGetLocks(ctx);
}
});
}
- public void postListLocks(final List<LockInfo> lockInfoList) throws IOException {
+ public void postGetLocks(final List<LockedResource> lockedResources) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
- oserver.postListLocks(ctx, lockInfoList);
+ oserver.postGetLocks(ctx, lockedResources);
}
});
}
@@ -1837,7 +1838,7 @@ public class MasterCoprocessorHost
}
public void preRequestLock(String namespace, TableName tableName, HRegionInfo[] regionInfos,
- LockProcedure.LockType type, String description) throws IOException {
+ LockType type, String description) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1848,7 +1849,7 @@ public class MasterCoprocessorHost
}
public void postRequestLock(String namespace, TableName tableName, HRegionInfo[] regionInfos,
- LockProcedure.LockType type, String description) throws IOException {
+ LockType type, String description) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
index d092efe..52b88db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
@@ -120,7 +120,7 @@ public class MasterMobCompactionThread {
public void run() {
// These locks are on dummy table names, and only used for compaction/mob file cleaning.
final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
- MobUtils.getTableLockName(tableName), LockProcedure.LockType.EXCLUSIVE,
+ MobUtils.getTableLockName(tableName), LockType.EXCLUSIVE,
this.getClass().getName() + ": mob compaction");
try {
for (ColumnFamilyDescriptor hcd : hcds) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 3ec2c45..971fa3b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@@ -60,8 +59,10 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
+import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
import org.apache.hadoop.hbase.quotas.QuotaUtil;
@@ -82,8 +83,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegi
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.*;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
@@ -92,8 +95,111 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockH
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
@@ -108,6 +214,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRe
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
@@ -116,10 +224,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRe
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
@@ -1078,13 +1186,13 @@ public class MasterRpcServices extends RSRpcServices
}
@Override
- public ListProceduresResponse listProcedures(
+ public GetProceduresResponse getProcedures(
RpcController rpcController,
- ListProceduresRequest request) throws ServiceException {
+ GetProceduresRequest request) throws ServiceException {
try {
- final ListProceduresResponse.Builder response = ListProceduresResponse.newBuilder();
- for (ProcedureInfo p: master.listProcedures()) {
- response.addProcedure(ProtobufUtil.toProtoProcedure(p));
+ final GetProceduresResponse.Builder response = GetProceduresResponse.newBuilder();
+ for (Procedure<?> p: master.getProcedures()) {
+ response.addProcedure(ProcedureUtil.convertToProtoProcedure(p));
}
return response.build();
} catch (IOException e) {
@@ -1093,14 +1201,14 @@ public class MasterRpcServices extends RSRpcServices
}
@Override
- public ListLocksResponse listLocks(
+ public GetLocksResponse getLocks(
RpcController controller,
- ListLocksRequest request) throws ServiceException {
+ GetLocksRequest request) throws ServiceException {
try {
- final ListLocksResponse.Builder builder = ListLocksResponse.newBuilder();
+ final GetLocksResponse.Builder builder = GetLocksResponse.newBuilder();
- for (LockInfo lockInfo: master.listLocks()) {
- builder.addLock(ProtobufUtil.toProtoLockInfo(lockInfo));
+ for (LockedResource lockedResource: master.getLocks()) {
+ builder.addLock(ProcedureUtil.convertToProtoLockedResource(lockedResource));
}
return builder.build();
@@ -1655,28 +1763,28 @@ public class MasterRpcServices extends RSRpcServices
SecurityCapabilitiesResponse.Builder response = SecurityCapabilitiesResponse.newBuilder();
try {
master.checkInitialized();
- Set<Capability> capabilities = new HashSet<>();
+ Set<SecurityCapabilitiesResponse.Capability> capabilities = new HashSet<>();
// Authentication
if (User.isHBaseSecurityEnabled(master.getConfiguration())) {
- capabilities.add(Capability.SECURE_AUTHENTICATION);
+ capabilities.add(SecurityCapabilitiesResponse.Capability.SECURE_AUTHENTICATION);
} else {
- capabilities.add(Capability.SIMPLE_AUTHENTICATION);
+ capabilities.add(SecurityCapabilitiesResponse.Capability.SIMPLE_AUTHENTICATION);
}
// The AccessController can provide AUTHORIZATION and CELL_AUTHORIZATION
if (master.cpHost != null &&
master.cpHost.findCoprocessor(AccessController.class.getName()) != null) {
if (AccessController.isAuthorizationSupported(master.getConfiguration())) {
- capabilities.add(Capability.AUTHORIZATION);
+ capabilities.add(SecurityCapabilitiesResponse.Capability.AUTHORIZATION);
}
if (AccessController.isCellAuthorizationSupported(master.getConfiguration())) {
- capabilities.add(Capability.CELL_AUTHORIZATION);
+ capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_AUTHORIZATION);
}
}
// The VisibilityController can provide CELL_VISIBILITY
if (master.cpHost != null &&
master.cpHost.findCoprocessor(VisibilityController.class.getName()) != null) {
if (VisibilityController.isCellAuthorizationSupported(master.getConfiguration())) {
- capabilities.add(Capability.CELL_VISIBILITY);
+ capabilities.add(SecurityCapabilitiesResponse.Capability.CELL_VISIBILITY);
}
}
response.addAllCapabilities(capabilities);
@@ -1846,7 +1954,7 @@ public class MasterRpcServices extends RSRpcServices
throw new IllegalArgumentException("Empty description");
}
NonceProcedureRunnable npr;
- LockProcedure.LockType type = LockProcedure.LockType.valueOf(request.getLockType().name());
+ LockType type = LockType.valueOf(request.getLockType().name());
if (request.getRegionInfoCount() > 0) {
final HRegionInfo[] regionInfos = new HRegionInfo[request.getRegionInfoCount()];
for (int i = 0; i < request.getRegionInfoCount(); ++i) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index cde9e34..6d5c53f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
@@ -41,7 +40,8 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
-import org.apache.hadoop.hbase.procedure2.LockInfo;
+import org.apache.hadoop.hbase.procedure2.LockedResource;
+import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
@@ -362,18 +362,18 @@ public interface MasterServices extends Server {
throws IOException;
/**
- * List procedures
+ * Get procedures
* @return procedure list
* @throws IOException
*/
- public List<ProcedureInfo> listProcedures() throws IOException;
+ public List<Procedure<?>> getProcedures() throws IOException;
/**
- * List locks
+ * Get locks
* @return lock list
* @throws IOException
*/
- public List<LockInfo> listLocks() throws IOException;
+ public List<LockedResource> getLocks() throws IOException;
/**
* Get list of table descriptors by namespace
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java
index 476c65c..2e4ff9a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java
@@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.procedure2.LockType;
/**
* The Class MobCompactChore for running compaction regularly to merge small mob files.
@@ -64,7 +64,7 @@ public class MobCompactionChore extends ScheduledChore {
boolean reported = false;
try {
final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
- MobUtils.getTableLockName(htd.getTableName()), LockProcedure.LockType.EXCLUSIVE,
+ MobUtils.getTableLockName(htd.getTableName()), LockType.EXCLUSIVE,
this.getClass().getName() + ": mob compaction");
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
if (!hcd.isMobEnabled()) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
index d78ba74..abc9796 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
@@ -20,9 +20,6 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
@@ -38,6 +35,7 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionOpenOperation;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -121,7 +119,8 @@ public class AssignProcedure extends RegionTransitionProcedure {
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
final AssignRegionStateData.Builder state = AssignRegionStateData.newBuilder()
.setTransitionState(getTransitionState())
.setRegionInfo(HRegionInfo.convert(getRegionInfo()));
@@ -131,12 +130,13 @@ public class AssignProcedure extends RegionTransitionProcedure {
if (this.targetServer != null) {
state.setTargetServer(ProtobufUtil.toServerName(this.targetServer));
}
- state.build().writeDelimitedTo(stream);
+ serializer.serialize(state.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- final AssignRegionStateData state = AssignRegionStateData.parseDelimitedFrom(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ final AssignRegionStateData state = serializer.deserialize(AssignRegionStateData.class);
setTransitionState(state.getTransitionState());
setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
forceNewPlan = state.getForceNewPlan();
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
index c7d97ee..27f6707 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
@@ -18,9 +18,6 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
@@ -29,6 +26,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -132,21 +130,23 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
}
@Override
- protected void serializeStateData(OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
final MasterProcedureProtos.GCMergedRegionsStateData.Builder msg =
MasterProcedureProtos.GCMergedRegionsStateData.newBuilder().
setParentA(HRegionInfo.convert(this.father)).
setParentB(HRegionInfo.convert(this.mother)).
setMergedChild(HRegionInfo.convert(this.mergedChild));
- msg.build().writeDelimitedTo(stream);
+ serializer.serialize(msg.build());
}
@Override
- protected void deserializeStateData(InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
final MasterProcedureProtos.GCMergedRegionsStateData msg =
- MasterProcedureProtos.GCMergedRegionsStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.GCMergedRegionsStateData.class);
this.father = HRegionInfo.convert(msg.getParentA());
this.mother = HRegionInfo.convert(msg.getParentB());
this.mergedChild = HRegionInfo.convert(msg.getMergedChild());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
index 29d0676..2d7f239 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
@@ -18,9 +18,6 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
@@ -32,6 +29,7 @@ import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -131,20 +129,22 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
}
@Override
- protected void serializeStateData(OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
// Double serialization of regionname. Superclass is also serializing. Fix.
final MasterProcedureProtos.GCRegionStateData.Builder msg =
MasterProcedureProtos.GCRegionStateData.newBuilder()
.setRegionInfo(HRegionInfo.convert(getRegion()));
- msg.build().writeDelimitedTo(stream);
+ serializer.serialize(msg.build());
}
@Override
- protected void deserializeStateData(InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
final MasterProcedureProtos.GCRegionStateData msg =
- MasterProcedureProtos.GCRegionStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.GCRegionStateData.class);
setRegion(HRegionInfo.convert(msg.getRegionInfo()));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index c398c9a..1741bd6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -53,13 +51,15 @@ import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
-import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
@@ -67,8 +67,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
/**
* The procedure to Merge a region in a table.
* This procedure takes an exclusive table lock since it is working over multiple regions.
@@ -346,8 +344,9 @@ public class MergeTableRegionsProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
final MasterProcedureProtos.MergeTableRegionsStateData.Builder mergeTableRegionsMsg =
MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
@@ -357,15 +356,16 @@ public class MergeTableRegionsProcedure
for (int i = 0; i < regionsToMerge.length; ++i) {
mergeTableRegionsMsg.addRegionInfo(HRegionInfo.convert(regionsToMerge[i]));
}
- mergeTableRegionsMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(mergeTableRegionsMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
final MasterProcedureProtos.MergeTableRegionsStateData mergeTableRegionsMsg =
- MasterProcedureProtos.MergeTableRegionsStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.MergeTableRegionsStateData.class);
setUser(MasterProcedureUtil.toUserInfo(mergeTableRegionsMsg.getUserInfo()));
assert(mergeTableRegionsMsg.getRegionInfoCount() == 2);
@@ -479,7 +479,7 @@ public class MergeTableRegionsProcedure
new IOException("Merge of " + regionsStr + " failed because merge switch is off"));
return false;
}
-
+
// Ask the remote regionserver if regions are mergeable. If we get an IOE, report it
// along w/ the failure so can see why we are not mergeable at this time.
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
index 1907e98..9e0d5f1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
@@ -20,8 +20,6 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -32,6 +30,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionStateData;
@@ -124,8 +123,9 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
final MoveRegionStateData.Builder state = MoveRegionStateData.newBuilder()
// No need to serialize the HRegionInfo. The super class has the region.
@@ -133,14 +133,16 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
if (plan.getDestination() != null) {
state.setDestinationServer(ProtobufUtil.toServerName(plan.getDestination()));
}
- state.build().writeDelimitedTo(stream);
+
+ serializer.serialize(state.build());
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
- final MoveRegionStateData state = MoveRegionStateData.parseDelimitedFrom(stream);
+ final MoveRegionStateData state = serializer.deserialize(MoveRegionStateData.class);
final HRegionInfo regionInfo = getRegion(); // Get it from super class deserialization.
final ServerName sourceServer = ProtobufUtil.toServerName(state.getSourceServer());
final ServerName destinationServer = state.hasDestinationServer() ?
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 8a26380..a81bbe1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -22,9 +22,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import java.io.IOException;
-import java.io.InputStream;
import java.io.InterruptedIOException;
-import java.io.OutputStream;
import java.util.*;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
@@ -39,7 +37,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
@@ -59,6 +56,7 @@ import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProced
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
@@ -323,8 +321,9 @@ public class SplitTableRegionProcedure
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
final MasterProcedureProtos.SplitTableRegionStateData.Builder splitTableRegionMsg =
MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
@@ -332,15 +331,16 @@ public class SplitTableRegionProcedure
.setParentRegionInfo(HRegionInfo.convert(getRegion()))
.addChildRegionInfo(HRegionInfo.convert(daughter_1_HRI))
.addChildRegionInfo(HRegionInfo.convert(daughter_2_HRI));
- splitTableRegionMsg.build().writeDelimitedTo(stream);
+ serializer.serialize(splitTableRegionMsg.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
final MasterProcedureProtos.SplitTableRegionStateData splitTableRegionsMsg =
- MasterProcedureProtos.SplitTableRegionStateData.parseDelimitedFrom(stream);
+ serializer.deserialize(MasterProcedureProtos.SplitTableRegionStateData.class);
setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
setRegion(HRegionInfo.convert(splitTableRegionsMsg.getParentRegionInfo()));
assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
@@ -678,6 +678,7 @@ public class SplitTableRegionProcedure
this.family = family;
}
+ @Override
public Pair<Path,Path> call() throws IOException {
return splitStoreFile(regionFs, family, sf);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
index c9f0fac..7d875b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
@@ -20,11 +20,6 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.ConnectException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
@@ -40,6 +35,7 @@ import org.apache.hadoop.hbase.master.procedure.ServerCrashException;
import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionCloseOperation;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
@@ -48,7 +44,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-
/**
* Procedure that describes the unassignment of a single region.
* There can only be one RegionTransitionProcedure -- i.e. an assign or an unassign -- per region
@@ -128,7 +123,8 @@ public class UnassignProcedure extends RegionTransitionProcedure {
}
@Override
- public void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
UnassignRegionStateData.Builder state = UnassignRegionStateData.newBuilder()
.setTransitionState(getTransitionState())
.setHostingServer(ProtobufUtil.toServerName(this.hostingServer))
@@ -139,12 +135,14 @@ public class UnassignProcedure extends RegionTransitionProcedure {
if (force) {
state.setForce(true);
}
- state.build().writeDelimitedTo(stream);
+ serializer.serialize(state.build());
}
@Override
- public void deserializeStateData(final InputStream stream) throws IOException {
- final UnassignRegionStateData state = UnassignRegionStateData.parseDelimitedFrom(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ final UnassignRegionStateData state =
+ serializer.deserialize(UnassignRegionStateData.class);
setTransitionState(state.getTransitionState());
setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
this.hostingServer = ProtobufUtil.toServerName(state.getHostingServer());
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
index 6c8bbba..87ad557d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
@@ -18,19 +18,20 @@
*/
package org.apache.hadoop.hbase.master.locking;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.NonceKey;
-import java.io.IOException;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
/**
* Functions to acquire lock on table/namespace/regions.
*/
@@ -50,12 +51,12 @@ public final class LockManager {
}
public MasterLock createMasterLock(final String namespace,
- final LockProcedure.LockType type, final String description) {
+ final LockType type, final String description) {
return new MasterLock(namespace, type, description);
}
public MasterLock createMasterLock(final TableName tableName,
- final LockProcedure.LockType type, final String description) {
+ final LockType type, final String description) {
return new MasterLock(tableName, type, description);
}
@@ -81,13 +82,13 @@ public final class LockManager {
private final String namespace;
private final TableName tableName;
private final HRegionInfo[] regionInfos;
- private final LockProcedure.LockType type;
+ private final LockType type;
private final String description;
private LockProcedure proc = null;
public MasterLock(final String namespace,
- final LockProcedure.LockType type, final String description) {
+ final LockType type, final String description) {
this.namespace = namespace;
this.tableName = null;
this.regionInfos = null;
@@ -96,7 +97,7 @@ public final class LockManager {
}
public MasterLock(final TableName tableName,
- final LockProcedure.LockType type, final String description) {
+ final LockType type, final String description) {
this.namespace = null;
this.tableName = tableName;
this.regionInfos = null;
@@ -108,7 +109,7 @@ public final class LockManager {
this.namespace = null;
this.tableName = null;
this.regionInfos = regionInfos;
- this.type = LockProcedure.LockType.EXCLUSIVE;
+ this.type = LockType.EXCLUSIVE;
this.description = description;
}
@@ -203,7 +204,7 @@ public final class LockManager {
* locks, regular heartbeats are required to keep the lock held.
*/
public class RemoteLocks {
- public long requestNamespaceLock(final String namespace, final LockProcedure.LockType type,
+ public long requestNamespaceLock(final String namespace, final LockType type,
final String description, final NonceKey nonceKey)
throws IllegalArgumentException, IOException {
master.getMasterCoprocessorHost().preRequestLock(namespace, null, null, type, description);
@@ -214,7 +215,7 @@ public final class LockManager {
return proc.getProcId();
}
- public long requestTableLock(final TableName tableName, final LockProcedure.LockType type,
+ public long requestTableLock(final TableName tableName, final LockType type,
final String description, final NonceKey nonceKey)
throws IllegalArgumentException, IOException {
master.getMasterCoprocessorHost().preRequestLock(null, tableName, null, type, description);
@@ -232,12 +233,12 @@ public final class LockManager {
final NonceKey nonceKey)
throws IllegalArgumentException, IOException {
master.getMasterCoprocessorHost().preRequestLock(null, null, regionInfos,
- LockProcedure.LockType.EXCLUSIVE, description);
+ LockType.EXCLUSIVE, description);
final LockProcedure proc = new LockProcedure(master.getConfiguration(), regionInfos,
- LockProcedure.LockType.EXCLUSIVE, description, null);
+ LockType.EXCLUSIVE, description, null);
submitProcedure(proc, nonceKey);
master.getMasterCoprocessorHost().postRequestLock(null, null, regionInfos,
- LockProcedure.LockType.EXCLUSIVE, description);
+ LockType.EXCLUSIVE, description);
return proc.getProcId();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
index edbba83..e7b4168 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
@@ -27,8 +27,10 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
@@ -36,8 +38,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockP
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
@@ -66,9 +66,6 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
public static final String LOCAL_MASTER_LOCKS_TIMEOUT_MS_CONF =
"hbase.master.procedure.local.master.locks.timeout.ms";
- // Also used in serialized states, changes will affect backward compatibility.
- public enum LockType { SHARED, EXCLUSIVE }
-
private String namespace;
private TableName tableName;
private HRegionInfo[] regionInfos;
@@ -265,7 +262,8 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
final LockProcedureData.Builder builder = LockProcedureData.newBuilder()
.setLockType(LockServiceProtos.LockType.valueOf(type.name()))
.setDescription(description);
@@ -281,12 +279,13 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
if (lockAcquireLatch != null) {
builder.setIsMasterLock(true);
}
- builder.build().writeDelimitedTo(stream);
+ serializer.serialize(builder.build());
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- final LockProcedureData state = LockProcedureData.parseDelimitedFrom(stream);
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ final LockProcedureData state = serializer.deserialize(LockProcedureData.class);
type = LockType.valueOf(state.getLockType().name());
description = state.getDescription();
if (state.getRegionInfoCount() > 0) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/359fed7b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
index 41502d4..c254cc4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
@@ -19,14 +19,12 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
/**
@@ -84,6 +82,7 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
* @param env MasterProcedureEnv
* @throws IOException
*/
+ @Override
protected void checkTableModifiable(final MasterProcedureEnv env) throws IOException {
// Checks whether the table exists
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), getTableName())) {
@@ -96,6 +95,7 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
return true;
}
+ @Override
protected LockState acquireLock(final MasterProcedureEnv env) {
if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
if (env.getProcedureScheduler().waitRegions(this, getTableName(), getRegion())) {
@@ -105,6 +105,7 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
return LockState.LOCK_ACQUIRED;
}
+ @Override
protected void releaseLock(final MasterProcedureEnv env) {
this.lock = false;
env.getProcedureScheduler().wakeRegions(this, getTableName(), getRegion());
@@ -120,14 +121,16 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
}
@Override
- protected void serializeStateData(final OutputStream stream) throws IOException {
- super.serializeStateData(stream);
- HRegionInfo.convert(getRegion()).writeDelimitedTo(stream);
+ protected void serializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.serializeStateData(serializer);
+ serializer.serialize(HRegionInfo.convert(getRegion()));
}
@Override
- protected void deserializeStateData(final InputStream stream) throws IOException {
- super.deserializeStateData(stream);
- this.hri = HRegionInfo.convert(HBaseProtos.RegionInfo.parseDelimitedFrom(stream));
+ protected void deserializeStateData(ProcedureStateSerializer serializer)
+ throws IOException {
+ super.deserializeStateData(serializer);
+ this.hri = HRegionInfo.convert(serializer.deserialize(HBaseProtos.RegionInfo.class));
}
}
\ No newline at end of file