hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r1333099 [5/5] - in /hbase/trunk/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/executor/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/protobuf/ main/java/org/apache/hadoop/hbase/pro...
Date Wed, 02 May 2012 16:26:38 GMT
Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java?rev=1333099&r1=1333098&r2=1333099&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java Wed
May  2 16:26:36 2012
@@ -19,27 +19,43 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
+import static org.apache.hadoop.hbase.SplitLogCounters.resetCounters;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_get_data_nonode;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_heartbeat;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_log_split_batch_success;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_node_create_queued;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_node_create_result;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_orphan_task_acquired;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_rescan;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_rescan_deleted;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_dead_server_task;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_failed;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_threshold_reached;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_unassigned;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_task_deleted;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
 
-import static org.junit.Assert.*;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SplitLogTask;
+import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.master.SplitLogManager.Task;
 import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
 import org.apache.hadoop.hbase.regionserver.TestMasterAddressManager.NodeCreationListener;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
-import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.log4j.Level;
@@ -57,6 +73,7 @@ import org.junit.experimental.categories
 @Category(MediumTests.class)
 public class TestSplitLogManager {
   private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
+  private final ServerName DUMMY_MASTER = new ServerName("dummy-master,1,1");
   static {
     Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
   }
@@ -125,8 +142,7 @@ public class TestSplitLogManager {
     public long eval();
   }
 
-  private void waitForCounter(final AtomicLong ctr, long oldval, long newval,
-      long timems) {
+  private void waitForCounter(final AtomicLong ctr, long oldval, long newval, long timems)
{
     Expr e = new Expr() {
       public long eval() {
         return ctr.get();
@@ -180,16 +196,16 @@ public class TestSplitLogManager {
   @Test
   public void testTaskCreation() throws Exception {
     LOG.info("TestTaskCreation - test the creation of a task in zk");
-
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     TaskBatch batch = new TaskBatch();
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
 
     byte[] data = ZKUtil.getData(zkw, tasknode);
-    LOG.info("Task node created " + new String(data));
-    assertTrue(TaskState.TASK_UNASSIGNED.equals(data, "dummy-master"));
+    SplitLogTask slt = SplitLogTask.parseFrom(data);
+    LOG.info("Task node created " + slt.toString());
+    assertTrue(slt.isUnassigned(DUMMY_MASTER));
   }
 
   @Test
@@ -197,8 +213,8 @@ public class TestSplitLogManager {
     LOG.info("TestOrphanTaskAcquisition");
 
     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
-    zkw.getRecoverableZooKeeper().create(tasknode,
-        TaskState.TASK_OWNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
+    SplitLogTask slt = new SplitLogTask.Owned(DUMMY_MASTER);
+    zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
         CreateMode.PERSISTENT);
 
     int to = 1000;
@@ -207,7 +223,7 @@ public class TestSplitLogManager {
     to = to + 2 * 100;
 
 
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
     Task task = slm.findOrCreateOrphanTask(tasknode);
@@ -229,12 +245,12 @@ public class TestSplitLogManager {
         " startup");
     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
     //create an unassigned orphan task
-    zkw.getRecoverableZooKeeper().create(tasknode,
-        TaskState.TASK_UNASSIGNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
+    SplitLogTask slt = new SplitLogTask.Unassigned(DUMMY_MASTER);
+    zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
         CreateMode.PERSISTENT);
     int version = ZKUtil.checkExists(zkw, tasknode);
 
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
     Task task = slm.findOrCreateOrphanTask(tasknode);
@@ -263,24 +279,29 @@ public class TestSplitLogManager {
     to = to + 2 * 100;
 
     conf.setInt("hbase.splitlog.max.resubmit", 2);
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     TaskBatch batch = new TaskBatch();
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
     int version = ZKUtil.checkExists(zkw, tasknode);
-
-    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
+    final ServerName worker1 = new ServerName("worker1,1,1");
+    final ServerName worker2 = new ServerName("worker2,1,1");
+    final ServerName worker3 = new ServerName("worker3,1,1");
+    SplitLogTask slt = new SplitLogTask.Owned(worker1);
+    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
     waitForCounter(tot_mgr_resubmit, 0, 1, to + EXTRA_TOLERANCE_MS);
     int version1 = ZKUtil.checkExists(zkw, tasknode);
     assertTrue(version1 > version);
-    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker2"));
+    slt = new SplitLogTask.Owned(worker2);
+    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     waitForCounter(tot_mgr_heartbeat, 1, 2, 1000);
     waitForCounter(tot_mgr_resubmit, 1, 2, to + 100);
     int version2 = ZKUtil.checkExists(zkw, tasknode);
     assertTrue(version2 > version1);
-    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker3"));
+    slt = new SplitLogTask.Owned(worker3);
+    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     waitForCounter(tot_mgr_heartbeat, 1, 2, 1000);
     waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + EXTRA_TOLERANCE_MS);
     Thread.sleep(to + EXTRA_TOLERANCE_MS);
@@ -293,14 +314,15 @@ public class TestSplitLogManager {
 
     conf.setInt("hbase.splitlog.manager.timeout", 1000);
     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     TaskBatch batch = new TaskBatch();
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
     int version = ZKUtil.checkExists(zkw, tasknode);
-
-    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
+    final ServerName worker1 = new ServerName("worker1,1,1");
+    SplitLogTask slt = new SplitLogTask.Owned(worker1);
+    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
     waitForCounter(new Expr() {
       @Override
@@ -312,8 +334,8 @@ public class TestSplitLogManager {
       int version1 = ZKUtil.checkExists(zkw, tasknode);
       assertTrue(version1 > version);
       byte[] taskstate = ZKUtil.getData(zkw, tasknode);
-      assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"),
-          taskstate));
+      slt = SplitLogTask.parseFrom(taskstate);
+      assertTrue(slt.isUnassigned(DUMMY_MASTER));
       
       waitForCounter(tot_mgr_rescan_deleted, 0, 1, 1000);
     } else {
@@ -327,11 +349,13 @@ public class TestSplitLogManager {
   public void testTaskDone() throws Exception {
     LOG.info("TestTaskDone - cleanup task node once in DONE state");
 
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     TaskBatch batch = new TaskBatch();
     String tasknode = submitTaskAndWait(batch, "foo/1");
-    ZKUtil.setData(zkw, tasknode, TaskState.TASK_DONE.get("worker"));
+    final ServerName worker1 = new ServerName("worker1,1,1");
+    SplitLogTask slt = new SplitLogTask.Done(worker1);
+    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     synchronized (batch) {
       while (batch.installed != batch.done) {
         batch.wait();
@@ -346,12 +370,14 @@ public class TestSplitLogManager {
     LOG.info("TestTaskErr - cleanup task node once in ERR state");
 
     conf.setInt("hbase.splitlog.max.resubmit", 0);
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     TaskBatch batch = new TaskBatch();
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
-    ZKUtil.setData(zkw, tasknode, TaskState.TASK_ERR.get("worker"));
+    final ServerName worker1 = new ServerName("worker1,1,1");
+    SplitLogTask slt = new SplitLogTask.Err(worker1);
+    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     synchronized (batch) {
       while (batch.installed != batch.error) {
         batch.wait();
@@ -359,27 +385,34 @@ public class TestSplitLogManager {
     }
     waitForCounter(tot_mgr_task_deleted, 0, 1, 1000);
     assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
-    conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLog.DEFAULT_MAX_RESUBMIT);
+    conf.setInt("hbase.splitlog.max.resubmit", SplitLogManager.DEFAULT_MAX_RESUBMIT);
   }
 
   @Test
   public void testTaskResigned() throws Exception {
     LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");
-
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    assertEquals(tot_mgr_resubmit.get(), 0);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
+    assertEquals(tot_mgr_resubmit.get(), 0);
     TaskBatch batch = new TaskBatch();
     String tasknode = submitTaskAndWait(batch, "foo/1");
-    ZKUtil.setData(zkw, tasknode, TaskState.TASK_RESIGNED.get("worker"));
+    assertEquals(tot_mgr_resubmit.get(), 0);
+    final ServerName worker1 = new ServerName("worker1,1,1");
+    assertEquals(tot_mgr_resubmit.get(), 0);
+    SplitLogTask slt = new SplitLogTask.Resigned(worker1);
+    assertEquals(tot_mgr_resubmit.get(), 0);
+    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     int version = ZKUtil.checkExists(zkw, tasknode);
-
-    waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
+    // Could be small race here.
+    if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
+    assertEquals(tot_mgr_resubmit.get(), 1);
     int version1 = ZKUtil.checkExists(zkw, tasknode);
-    assertTrue(version1 > version);
+    assertTrue("version1=" + version1 + ", version=" + version, version1 > version);
 
     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
-    assertTrue(Arrays.equals(taskstate,
-        TaskState.TASK_UNASSIGNED.get("dummy-master")));
+    slt = SplitLogTask.parseFrom(taskstate);
+    assertTrue(slt.isUnassigned(DUMMY_MASTER));
   }
 
   @Test
@@ -389,8 +422,9 @@ public class TestSplitLogManager {
 
     // create an orphan task in OWNED state
     String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
-    zkw.getRecoverableZooKeeper().create(tasknode1,
-        TaskState.TASK_OWNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
+    final ServerName worker1 = new ServerName("worker1,1,1");
+    SplitLogTask slt = new SplitLogTask.Owned(worker1);
+    zkw.getRecoverableZooKeeper().create(tasknode1, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
         CreateMode.PERSISTENT);
 
     int to = 1000;
@@ -399,7 +433,7 @@ public class TestSplitLogManager {
     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
 
 
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
 
@@ -411,8 +445,9 @@ public class TestSplitLogManager {
     // keep updating the orphan owned node every to/2 seconds
     for (int i = 0; i < (3 * to)/100; i++) {
       Thread.sleep(100);
-      ZKUtil.setData(zkw, tasknode1,
-          TaskState.TASK_OWNED.get("dummy-worker"));
+      final ServerName worker2 = new ServerName("worker1,1,1");
+      slt = new SplitLogTask.Owned(worker2);
+      ZKUtil.setData(zkw, tasknode1, slt.toByteArray());
     }
 
     // since we have stopped heartbeating the owned node therefore it should
@@ -429,31 +464,34 @@ public class TestSplitLogManager {
     LOG.info("testDeadWorker");
 
     conf.setLong("hbase.splitlog.max.resubmit", 0);
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     TaskBatch batch = new TaskBatch();
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
     int version = ZKUtil.checkExists(zkw, tasknode);
-
-    ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
-    waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
-    slm.handleDeadWorker("worker1");
-    waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
-    waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, 1000);
+    final ServerName worker1 = new ServerName("worker1,1,1");
+    SplitLogTask slt = new SplitLogTask.Owned(worker1);
+    ZKUtil.setData(zkw, tasknode, slt.toByteArray());
+    if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
+    slm.handleDeadWorker(worker1);
+    if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, 3000);
+    if (tot_mgr_resubmit_dead_server_task.get() == 0) {
+      waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, 3000);
+    }
 
     int version1 = ZKUtil.checkExists(zkw, tasknode);
     assertTrue(version1 > version);
     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
-    assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"),
-        taskstate));
+    slt = SplitLogTask.parseFrom(taskstate);
+    assertTrue(slt.isUnassigned(DUMMY_MASTER));
     return;
   }
 
   @Test
   public void testEmptyLogDir() throws Exception {
     LOG.info("testEmptyLogDir");
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     Path emptyLogDirPath = new Path(fs.getWorkingDirectory(),
@@ -467,7 +505,7 @@ public class TestSplitLogManager {
   public void testVanishingTaskZNode() throws Exception {
     LOG.info("testVanishingTaskZNode");
     conf.setInt("hbase.splitlog.manager.unassigned.timeout", 0);
-    slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
+    slm = new SplitLogManager(zkw, conf, stopper, DUMMY_MASTER, null);
     slm.finishInitialization();
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     final Path logDir = new Path(fs.getWorkingDirectory(),

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java?rev=1333099&r1=1333098&r2=1333099&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
(original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
Wed May  2 16:26:36 2012
@@ -19,12 +19,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.resetCounters;
-import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_failed_to_grab_task_lost_race;
-import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_failed_to_grab_task_owned;
-import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_preempt_task;
-import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired;
-import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired_rescan;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -35,10 +29,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SplitLogCounters;
+import org.apache.hadoop.hbase.SplitLogTask;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
-import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.log4j.Level;
@@ -53,6 +48,7 @@ import org.junit.experimental.categories
 @Category(MediumTests.class)
 public class TestSplitLogWorker {
   private static final Log LOG = LogFactory.getLog(TestSplitLogWorker.class);
+  private final ServerName MANAGER = new ServerName("manager,1,1");
   static {
     Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
   }
@@ -98,7 +94,7 @@ public class TestSplitLogWorker {
     ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
     assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
     LOG.debug(zkw.splitLogZNode + " created");
-    resetCounters();
+    SplitLogCounters.resetCounters();
 
   }
 
@@ -129,19 +125,20 @@ public class TestSplitLogWorker {
   @Test
   public void testAcquireTaskAtStartup() throws Exception {
     LOG.info("testAcquireTaskAtStartup");
-    ZKSplitLog.Counters.resetCounters();
-
-    zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tatas"),
-        TaskState.TASK_UNASSIGNED.get("mgr"), Ids.OPEN_ACL_UNSAFE,
+    SplitLogCounters.resetCounters();
+    final String TATAS = "tatas";
+    final ServerName RS = new ServerName("rs,1,1");
+    zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TATAS),
+      new SplitLogTask.Unassigned(new ServerName("mgr,1,1")).toByteArray(), Ids.OPEN_ACL_UNSAFE,
         CreateMode.PERSISTENT);
 
-    SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
-      "rs", neverEndingTask);
+    SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), RS, neverEndingTask);
     slw.start();
     try {
-      waitForCounter(tot_wkr_task_acquired, 0, 1, 100);
-      assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
-        ZKSplitLog.getEncodedNodeName(zkw, "tatas")), "rs"));
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 100);
+      byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS));
+      SplitLogTask slt = SplitLogTask.parseFrom(bytes);
+      assertTrue(slt.isOwned(RS));
     } finally {
       stopSplitLogWorker(slw);
     }
@@ -161,28 +158,27 @@ public class TestSplitLogWorker {
   @Test
   public void testRaceForTask() throws Exception {
     LOG.info("testRaceForTask");
-    ZKSplitLog.Counters.resetCounters();
-
-    zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "trft"),
-        TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
+    SplitLogCounters.resetCounters();
+    final String TRFT = "trft";
+    final ServerName SVR1 = new ServerName("svr1,1,1");
+    final ServerName SVR2 = new ServerName("svr2,1,1");
+    zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TRFT),
+      new SplitLogTask.Unassigned(MANAGER).toByteArray(), Ids.OPEN_ACL_UNSAFE,
         CreateMode.PERSISTENT);
 
-    SplitLogWorker slw1 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
-        "svr1", neverEndingTask);
-    SplitLogWorker slw2 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
-        "svr2", neverEndingTask);
+    SplitLogWorker slw1 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SVR1, neverEndingTask);
+    SplitLogWorker slw2 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SVR2, neverEndingTask);
     slw1.start();
     slw2.start();
     try {
-      waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
       // Assert that either the tot_wkr_failed_to_grab_task_owned count was set of if
       // not it, that we fell through to the next counter in line and it was set.
-      assertTrue(waitForCounterBoolean(tot_wkr_failed_to_grab_task_owned, 0, 1, 1000) ||
-        tot_wkr_failed_to_grab_task_lost_race.get() == 1);
-      assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
-        ZKSplitLog.getEncodedNodeName(zkw, "trft")), "svr1") ||
-        TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
-            ZKSplitLog.getEncodedNodeName(zkw, "trft")), "svr2"));
+      assertTrue(waitForCounterBoolean(SplitLogCounters.tot_wkr_failed_to_grab_task_owned,
0, 1, 1000) ||
+          SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.get() == 1);
+      byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TRFT));
+      SplitLogTask slt = SplitLogTask.parseFrom(bytes);
+      assertTrue(slt.isOwned(SVR1) || slt.isOwned(SVR2));
     } finally {
       stopSplitLogWorker(slw1);
       stopSplitLogWorker(slw2);
@@ -192,28 +188,28 @@ public class TestSplitLogWorker {
   @Test
   public void testPreemptTask() throws Exception {
     LOG.info("testPreemptTask");
-    ZKSplitLog.Counters.resetCounters();
-
-    SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
-        "tpt_svr", neverEndingTask);
+    SplitLogCounters.resetCounters();
+    final ServerName SRV = new ServerName("tpt_svr,1,1");
+    final String PATH = ZKSplitLog.getEncodedNodeName(zkw, "tpt_task");
+    SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SRV, neverEndingTask);
     slw.start();
     try {
       Thread.yield(); // let the worker start
       Thread.sleep(100);
 
       // this time create a task node after starting the splitLogWorker
-      zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tpt_task"),
-        TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
+      zkw.getRecoverableZooKeeper().create(PATH,
+        new SplitLogTask.Unassigned(MANAGER).toByteArray(), Ids.OPEN_ACL_UNSAFE,
         CreateMode.PERSISTENT);
 
-      waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
       assertEquals(1, slw.taskReadySeq);
-      assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
-        ZKSplitLog.getEncodedNodeName(zkw, "tpt_task")), "tpt_svr"));
-
-      ZKUtil.setData(zkw, ZKSplitLog.getEncodedNodeName(zkw, "tpt_task"),
-        TaskState.TASK_UNASSIGNED.get("manager"));
-      waitForCounter(tot_wkr_preempt_task, 0, 1, 1000);
+      byte [] bytes = ZKUtil.getData(zkw, PATH);
+      SplitLogTask slt = SplitLogTask.parseFrom(bytes);
+      assertTrue(slt.isOwned(SRV));
+      slt = new SplitLogTask.Unassigned(MANAGER);
+      ZKUtil.setData(zkw, PATH, slt.toByteArray());
+      waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
     } finally {
       stopSplitLogWorker(slw);
     }
@@ -222,35 +218,37 @@ public class TestSplitLogWorker {
   @Test
   public void testMultipleTasks() throws Exception {
     LOG.info("testMultipleTasks");
-    ZKSplitLog.Counters.resetCounters();
-    SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
-        "tmt_svr", neverEndingTask);
+    SplitLogCounters.resetCounters();
+    final ServerName SRV = new ServerName("tmt_svr,1,1");
+    final String PATH1 = ZKSplitLog.getEncodedNodeName(zkw, "tmt_task");
+    SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SRV, neverEndingTask);
     slw.start();
     try {
       Thread.yield(); // let the worker start
       Thread.sleep(100);
+      SplitLogTask unassignedManager = new SplitLogTask.Unassigned(MANAGER);
+      zkw.getRecoverableZooKeeper().create(PATH1, unassignedManager.toByteArray(),
+        Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 
-      zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tmt_task"),
-        TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
-        CreateMode.PERSISTENT);
-
-      waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
       // now the worker is busy doing the above task
 
       // create another task
-      zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2"),
-        TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
-        CreateMode.PERSISTENT);
+      final String PATH2 = ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2");
+      zkw.getRecoverableZooKeeper().create(PATH2, unassignedManager.toByteArray(),
+        Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 
       // preempt the first task, have it owned by another worker
-      ZKUtil.setData(zkw, ZKSplitLog.getEncodedNodeName(zkw, "tmt_task"),
-        TaskState.TASK_OWNED.get("another-worker"));
-      waitForCounter(tot_wkr_preempt_task, 0, 1, 1000);
+      final ServerName anotherWorker = new ServerName("another-worker,1,1");
+      SplitLogTask slt = new SplitLogTask.Owned(anotherWorker);
+      ZKUtil.setData(zkw, PATH1, slt.toByteArray());
+      waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
 
-      waitForCounter(tot_wkr_task_acquired, 1, 2, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, 1000);
       assertEquals(2, slw.taskReadySeq);
-      assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
-        ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2")), "tmt_svr"));
+      byte [] bytes = ZKUtil.getData(zkw, PATH2);
+      slt = SplitLogTask.parseFrom(bytes);
+      assertTrue(slt.isOwned(SRV));
     } finally {
       stopSplitLogWorker(slw);
     }
@@ -259,38 +257,37 @@ public class TestSplitLogWorker {
   @Test
   public void testRescan() throws Exception {
     LOG.info("testRescan");
-    ZKSplitLog.Counters.resetCounters();
-    slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
-        "svr", neverEndingTask);
+    SplitLogCounters.resetCounters();
+    final ServerName SRV = new ServerName("svr,1,1");
+    slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), SRV, neverEndingTask);
     slw.start();
     Thread.yield(); // let the worker start
     Thread.sleep(100);
 
     String task = ZKSplitLog.getEncodedNodeName(zkw, "task");
-    zkw.getRecoverableZooKeeper().create(task,
-      TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
+    SplitLogTask slt = new SplitLogTask.Unassigned(MANAGER);
+    zkw.getRecoverableZooKeeper().create(task,slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
       CreateMode.PERSISTENT);
 
-    waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
+    waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
     // now the worker is busy doing the above task
 
     // preempt the task, have it owned by another worker
-    ZKUtil.setData(zkw, task, TaskState.TASK_UNASSIGNED.get("manager"));
-    waitForCounter(tot_wkr_preempt_task, 0, 1, 1000);
+    ZKUtil.setData(zkw, task, slt.toByteArray());
+    waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
 
     // create a RESCAN node
     String rescan = ZKSplitLog.getEncodedNodeName(zkw, "RESCAN");
-    rescan = zkw.getRecoverableZooKeeper().create(rescan,
-      TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
+    rescan = zkw.getRecoverableZooKeeper().create(rescan, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
       CreateMode.PERSISTENT_SEQUENTIAL);
 
-    waitForCounter(tot_wkr_task_acquired, 1, 2, 1000);
+    waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, 1000);
     // RESCAN node might not have been processed if the worker became busy
     // with the above task. preempt the task again so that now the RESCAN
     // node is processed
-    ZKUtil.setData(zkw, task, TaskState.TASK_UNASSIGNED.get("manager"));
-    waitForCounter(tot_wkr_preempt_task, 1, 2, 1000);
-    waitForCounter(tot_wkr_task_acquired_rescan, 0, 1, 1000);
+    ZKUtil.setData(zkw, task, slt.toByteArray());
+    waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 1, 2, 1000);
+    waitForCounter(SplitLogCounters.tot_wkr_task_acquired_rescan, 0, 1, 1000);
 
     List<String> nodes = ZKUtil.listChildrenNoWatch(zkw, zkw.splitLogZNode);
     LOG.debug(nodes);
@@ -301,8 +298,8 @@ public class TestSplitLogWorker {
         String name = ZKSplitLog.getEncodedNodeName(zkw, node);
         String fn = ZKSplitLog.getFileName(name);
         byte [] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(zkw.splitLogZNode, fn));
-        String datastr = Bytes.toString(data);
-        assertTrue("data=" + datastr, TaskState.TASK_DONE.equals(data, "svr"));
+        slt = SplitLogTask.parseFrom(data);
+        assertTrue(slt.toString(), slt.isDone(SRV));
       }
     }
     assertEquals(2, num);
@@ -311,5 +308,4 @@ public class TestSplitLogWorker {
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
-}
-
+}
\ No newline at end of file

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java?rev=1333099&r1=1333098&r2=1333099&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
(original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
Wed May  2 16:26:36 2012
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.De
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
-import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@@ -98,9 +97,10 @@ public class TestSplitTransactionOnClust
    * @throws InterruptedException
    * @throws NodeExistsException
    * @throws KeeperException
+   * @throws DeserializationException 
    */
   @Test (timeout = 300000) public void testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling()
-  throws IOException, InterruptedException, NodeExistsException, KeeperException {
+  throws IOException, InterruptedException, NodeExistsException, KeeperException, DeserializationException
{
     final byte [] tableName =
       Bytes.toBytes("ephemeral");
 
@@ -137,12 +137,12 @@ public class TestSplitTransactionOnClust
       Stat stats =
         TESTING_UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
       LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats=" + stats);
-      RegionTransitionData rtd =
-        ZKAssign.getData(TESTING_UTIL.getZooKeeperWatcher(),
-          hri.getEncodedName());
+      RegionTransition rt =
+        RegionTransition.parseFrom(ZKAssign.getData(TESTING_UTIL.getZooKeeperWatcher(),
+          hri.getEncodedName()));
       // State could be SPLIT or SPLITTING.
-      assertTrue(rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLIT) ||
-        rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
+      assertTrue(rt.getEventType().equals(EventType.RS_ZK_REGION_SPLIT) ||
+        rt.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
       // Now crash the server
       cluster.abortRegionServer(tableRegionIndex);
       waitUntilRegionServerDead();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java?rev=1333099&r1=1333098&r2=1333099&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
(original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
Wed May  2 16:26:36 2012
@@ -25,14 +25,15 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.RegionTransition;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
-import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -133,9 +134,10 @@ public class TestCloseRegionHandler {
       * @throws IOException
       * @throws NodeExistsException
       * @throws KeeperException
+     * @throws DeserializationException 
       */
      @Test public void testZKClosingNodeVersionMismatch()
-     throws IOException, NodeExistsException, KeeperException {
+     throws IOException, NodeExistsException, KeeperException, DeserializationException {
        final Server server = new MockServer(HTU);
        final MockRegionServerServices rss = new MockRegionServerServices();
        rss.setFileSystem(HTU.getTestFileSystem());
@@ -160,9 +162,9 @@ public class TestCloseRegionHandler {
        handler.process();
    
        // Handler should remain in M_ZK_REGION_CLOSING
-       RegionTransitionData data =
-         ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName());
-       assertTrue(EventType.M_ZK_REGION_CLOSING == data.getEventType());
+       RegionTransition rt =
+         RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
+       assertTrue(rt.getEventType().equals(EventType.M_ZK_REGION_CLOSING ));
      }
   
      /**
@@ -170,9 +172,10 @@ public class TestCloseRegionHandler {
       * @throws IOException
       * @throws NodeExistsException
       * @throws KeeperException
+     * @throws DeserializationException 
       */
      @Test public void testCloseRegion()
-     throws IOException, NodeExistsException, KeeperException {
+     throws IOException, NodeExistsException, KeeperException, DeserializationException {
        final Server server = new MockServer(HTU);
        final MockRegionServerServices rss = new MockRegionServerServices();
        rss.setFileSystem(HTU.getTestFileSystem());
@@ -196,26 +199,23 @@ public class TestCloseRegionHandler {
          versionOfClosingNode);
        handler.process();
        // Handler should have transitioned it to RS_ZK_REGION_CLOSED
-       RegionTransitionData data =
-         ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName());
-       assertTrue(EventType.RS_ZK_REGION_CLOSED == data.getEventType());
+       RegionTransition rt = RegionTransition.parseFrom(
+         ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
+       assertTrue(rt.getEventType().equals(EventType.RS_ZK_REGION_CLOSED));
      }
+
      private void OpenRegion(Server server, RegionServerServices rss,
-           HTableDescriptor htd, HRegionInfo hri)
-           throws IOException, NodeExistsException, KeeperException {
-           // Create it OFFLINE node, which is what Master set before sending OPEN RPC
-           ZKAssign.createNodeOffline(server.getZooKeeper(), hri,
-             server.getServerName());
-           OpenRegionHandler openHandler = new OpenRegionHandler(server, rss, hri,
-             htd);
-           openHandler.process();
-           RegionTransitionData data =
-             ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName());
-       
-           // delete the node, which is what Master do after the region is opened
-           ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(),
-             EventType.RS_ZK_REGION_OPENED);
-         }  
+         HTableDescriptor htd, HRegionInfo hri)
+     throws IOException, NodeExistsException, KeeperException, DeserializationException {
+       // Create it OFFLINE node, which is what Master set before sending OPEN RPC
+       ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
+       OpenRegionHandler openHandler = new OpenRegionHandler(server, rss, hri, htd);
+       openHandler.process();
+       // This parse is not used?
+       RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
+       // delete the node, which is what Master do after the region is opened
+       ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(), EventType.RS_ZK_REGION_OPENED);
+     }  
 
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java?rev=1333099&r1=1333098&r2=1333099&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
(original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
Wed May  2 16:26:36 2012
@@ -26,7 +26,6 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -148,9 +147,9 @@ public class TestOpenRegionHandler {
     handler.process();
 
     // Handler should have transitioned it to FAILED_OPEN
-    RegionTransitionData data =
-      ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName());
-    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, data.getEventType());
+    RegionTransition rt = RegionTransition.parseFrom(
+      ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName()));
+    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
   }
   
   @Test
@@ -173,9 +172,9 @@ public class TestOpenRegionHandler {
     handler.process();
 
     // Handler should have transitioned it to FAILED_OPEN
-    RegionTransitionData data =
-      ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName());
-    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, data.getEventType());
+    RegionTransition rt = RegionTransition.parseFrom(
+      ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName()));
+    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
   }
   
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java?rev=1333099&r1=1333098&r2=1333099&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java Wed May  2 16:26:36
2012
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.RegionTransition;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.AdminProtocol;
 import org.apache.hadoop.hbase.client.Delete;
@@ -59,7 +60,6 @@ import org.apache.hadoop.hbase.client.Re
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
-import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -920,9 +920,9 @@ public class TestHBaseFsck {
 
       int iTimes = 0;
       while (true) {
-        RegionTransitionData rtd = ZKAssign.getData(zkw,
-            region.getEncodedName());
-        if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
+        RegionTransition rt = RegionTransition.parseFrom(ZKAssign.getData(zkw,
+            region.getEncodedName()));
+        if (rt != null && rt.getEventType() == EventType.RS_ZK_REGION_OPENED) {
           break;
         }
         Thread.sleep(100);



Mime
View raw message