You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/04/08 20:18:38 UTC

svn commit: r1585805 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/util/RollingRestart.java test/java/org/apache/hadoop/hbase/util/TestDrainRegionServer.java

Author: liyin
Date: Tue Apr  8 18:18:37 2014
New Revision: 1585805

URL: http://svn.apache.org/r1585805
Log:
[HBASE-10927] #3909589 RollingRestart.java to report the successfully drained regions + undrain onlySE-10927

Author: edunov

Summary:
There are couple of options now. You can specify drain log file via -location option or you can omit it and temp file will be created for you which you can use later

Note, that rolling_restart_v2 change is implemented but I was unable to test it because groovy version on hbase host is 1.8 and it has incompatible version of asm 3.2. Need to upgrade groovy on hbase host to 2.2 first.

Test Plan:
1. On hbase cluster run

./hbase org.apache.hadoop.hbase.util.RollingRestart -drain -location /tmp/drain.log.bin  -s hbase2464.ash3.facebook.com -p 60021
then
./hbase org.apache.hadoop.hbase.util.RollingRestart -undrain -location /tmp/drain.log.bin  -s <region server host> -p 60021

check the file  /tmp/drain.log.bin it should have regions in binary form

Dump all regions from the drain log file:
./hbase org.apache.hadoop.hbase.util.RollingRestart -readlog -location /tmp/drain.log.bin

Should print something similar to this:

14/04/03 11:39:32 INFO util.RollingRestart: Regions contained in the file:
t1,99999999,1396299107459.103fda0eed9b2b9a55421bc2ca9175ae.
t1,55555555,1396299107458.177065759563bf26d8069bfb88d8110e.
t1,11111111,1396299107458.86733e9549ada628073cd7a806ffa307.
t1,dddddddd,1396299107459.ff55239c7b55204a2962f61b8be4016a.

Reviewers: rshroff, rmk

Reviewed By: rshroff

CC: hbase-eng@, aleksandr

Differential Revision: https://phabricator.fb.com/D1256045

Task ID: 3909589

Added:
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestDrainRegionServer.java
Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RollingRestart.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RollingRestart.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RollingRestart.java?rev=1585805&r1=1585804&r2=1585805&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RollingRestart.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RollingRestart.java Tue Apr  8 18:18:37 2014
@@ -1,6 +1,10 @@
 package org.apache.hadoop.hbase.util;
 
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -69,6 +73,17 @@ public class RollingRestart {
       int sleepIntervalBeforeRestart, int getOpFrequency,
       boolean useHadoopCtl, int port) throws IOException {
 
+    this(serverName, regionDrainInterval, regionUndrainInterval,
+        sleepIntervalAfterRestart, sleepIntervalBeforeRestart, getOpFrequency,
+        useHadoopCtl, port, HBaseConfiguration.create());
+
+  }
+
+  RollingRestart(String serverName, int regionDrainInterval,
+                 int regionUndrainInterval, int sleepIntervalAfterRestart,
+                 int sleepIntervalBeforeRestart, int getOpFrequency,
+                 boolean useHadoopCtl, int port, Configuration conf) throws IOException {
+
     this.sleepIntervalAfterRestart = sleepIntervalAfterRestart;
     this.sleepIntervalBeforeRestart = sleepIntervalBeforeRestart;
     this.useHadoopCtl = useHadoopCtl;
@@ -77,7 +92,7 @@ public class RollingRestart {
     this.getOpFrequency = getOpFrequency;
     this.port = port;
 
-    conf = HBaseConfiguration.create();
+    this.conf = conf;
     this.moveRetries = conf.getInt("hbase.rollingrestart.move.maxretries", DEFAULT_MOVE_RETRIES);
     this.moveTimeoutInterval = conf.getInt("hbase.rollingrestart.move.timeout", DEFAULT_MOVE_TIMEOUT);
 
@@ -108,7 +123,7 @@ public class RollingRestart {
     UNDRAIN,
     COMPLETE,
     FAIL
-  };
+  }
 
   boolean moveRegion(final HRegionInfo region) throws Exception {
     HRegionInterface destinationServer = getDestinationServer(region);
@@ -159,13 +174,12 @@ public class RollingRestart {
   }
 
   /**
-   * Restarts the regionserver using the hadoopctl script. This adds
+   * Stops the regionserver using the hadoopctl script. This adds
    * a dependency on the hadoopctl script.
    * @throws IOException
    * @throws InterruptedException
-   * @param drainAndStopOnly
    */
-  void restart(boolean drainAndStopOnly) throws IOException, InterruptedException {
+  private void stop() throws InterruptedException, IOException {
     System.out.println("Shutting down the region server after sleep of " +
         this.sleepIntervalBeforeRestart);
     Thread.sleep(this.sleepIntervalBeforeRestart);
@@ -186,10 +200,39 @@ public class RollingRestart {
           LOG.error("Failed to stop regionserver. Aborting..");
           throw new IOException("Failed to stop regionserver. Aborting..");
         }
-        if(drainAndStopOnly) {
-          LOG.info("Only told to stop the region server. Returning..");
-          return;
+      } else {
+        String sshCmdToStopRS = sshCmd + " /usr/local/hadoop/" +
+            cellName + "-HBASE/bin/hbase-daemon.sh stop regionserver";
+        LOG.info("Executing " + sshCmd);
+        Process p = Runtime.getRuntime().exec(sshCmdToStopRS);
+        p.waitFor();
+
+        LOG.info("Exit value for the region server stop " + p.exitValue());
+
+        if (p.exitValue() != 0) {
+          LOG.error("Failed to stop regionserver. Aborting..");
+          throw new IOException("Failed to stop regionserver. Aborting..");
         }
+      }
+    } catch (IOException e) {
+      LOG.error("Unable to stop regionserver", e);
+      throw e;
+    }
+  }
+
+  /**
+   * Restarts (stops and then starts) the regionserver using the hadoopctl script. This adds
+   * a dependency on the hadoopctl script.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  void restart() throws IOException, InterruptedException {
+    stop();
+    String cellName = conf.get("titan.cell.name");
+    String sshCmd = "ssh hadoop@" + serverAddr.getHostname();
+
+    try {
+      if (this.useHadoopCtl) {
 
         String sshCmdToStartRS = sshCmd + " hadoopctl start regionserver";
         LOG.info("Executing " + sshCmdToStartRS);
@@ -205,21 +248,10 @@ public class RollingRestart {
         }
 
       } else {
-        String sshCmdToStopRS = sshCmd + " /usr/local/hadoop/" +
-            cellName + "-HBASE/bin/hbase-daemon.sh stop regionserver";
-        LOG.info("Executing " + sshCmd);
-        Process p = Runtime.getRuntime().exec(sshCmdToStopRS);
-        p.waitFor();
-
-        LOG.info("Exit value for the region server stop " + p.exitValue());
 
-        if (p.exitValue() != 0) {
-          LOG.error("Failed to stop regionserver. Aborting..");
-          throw new IOException("Failed to stop regionserver. Aborting..");
-        }
         String sshCmdToStartRS = sshCmd + " /usr/local/hadoop/" +
             cellName + "-HBASE/bin/hbase-daemon.sh start regionserver ";
-        p = Runtime.getRuntime().exec(sshCmdToStartRS);
+        Process p = Runtime.getRuntime().exec(sshCmdToStartRS);
         p.waitFor();
 
         LOG.info("Exit value for the region server start " + p.exitValue());
@@ -238,7 +270,7 @@ public class RollingRestart {
     // Wait for it to come back online
     while(true) {
       try {
-        if (getHRegionConnection(serverAddr).isStopped() == false) {
+        if (!getHRegionConnection(serverAddr).isStopped()) {
           break;
         }
      } catch (Exception e) {
@@ -282,26 +314,40 @@ public class RollingRestart {
     return null;
   }
 
-  void drainServer() throws Exception {
+  void drainServer(File drainedLogFile) throws Exception {
 
     LOG.info("Draining region server");
 
-    currentState = STAGE.DRAIN;
-    for (HRegionInfo region : regions) {
-      if (region.isMetaRegion() ||
-          region.isRootRegion()  ||
-          region.getRegionNameAsString().contains(",,")) {
-        continue;
-      }
-      if (!moveRegion(region)) {
-        throw new IOException("Failed to move region " + region.getRegionNameAsString() + ". Aborting");
+    DataOutputStream drainedRegionsLog = new DataOutputStream(
+        new FileOutputStream(drainedLogFile));
+    try {
+      currentState = STAGE.DRAIN;
+      for (HRegionInfo region : regions) {
+        if (region.isMetaRegion() ||
+            region.isRootRegion() ||
+            region.getRegionNameAsString().contains(",,")) {
+          continue;
+        }
+        if (moveRegion(region)) {
+          region.write(drainedRegionsLog);
+        } else {
+          LOG.error("Failed to move region " + region.getRegionNameAsString());
+        }
+        Thread.sleep(this.regionDrainInterval);
       }
-      Thread.sleep(this.regionDrainInterval);
+    } finally {
+      drainedRegionsLog.close();
     }
+
+    LOG.info("Draining complete. Drain log is written to [" + drainedLogFile.getCanonicalPath() + "]");
   }
 
-  void undrainServer() throws Exception {
+  void undrainServer(File inputFile) throws Exception {
     LOG.info("Undraining region server");
+    if (inputFile != null) {
+      regions = readDrainFile(inputFile);
+    }
+
     currentState = STAGE.UNDRAIN;
     for (HRegionInfo region : regions) {
       if (region.isMetaRegion() ||
@@ -310,7 +356,7 @@ public class RollingRestart {
         continue;
       }
       if (!moveRegion(region)) {
-        throw new IOException("Failed to move region " + region.getRegionNameAsString() + ". Aborting");
+        LOG.error("Failed to move region " + region.getRegionNameAsString());
       }
       Thread.sleep(this.regionUndrainInterval);
     }
@@ -339,26 +385,23 @@ public class RollingRestart {
     LOG.info("Setup Complete");
   }
 
-  public void clear(boolean drainAndRestartOnly) {
-
+  private void clear() {
     for (RegionChecker r : this.regionCheckers) {
       r.stop();
       r.printInfo();
     }
     this.regionCheckers.clear();
+  }
 
-    if (drainAndRestartOnly) {
-      LOG.warn("Not removing the regionserver from the blacklist.");
-      return;
-    }
-
+  private void removeFromBlackList() {
     try {
       admin.getMaster().clearBlacklistedServer(
           getHRegionConnection(serverAddr).getHServerInfo().getHostnamePort());
     } catch (IOException e) {
       LOG.error("Failed to remove the server from black list. Please remove it");
     }
-   }
+  }
+
 
    public static void clearAll() {
      Configuration conf = HBaseConfiguration.create();
@@ -401,7 +444,6 @@ public class RollingRestart {
          table = new HTable(conf, tableName);
        } catch (IOException e) {
          e.printStackTrace();
-         return;
        }
      }
 
@@ -451,7 +493,7 @@ public class RollingRestart {
        LOG.info(regionInfo.getRegionNameAsString() +
            ": total timeout = " + totalTimeout + ", number of errors = " +  errors.size());
      }
-   };
+   }
 
   /**
    * @param args
@@ -483,6 +525,14 @@ public class RollingRestart {
       "Drain and stop the region server(Works only with hadoopctl). Default : false");
     options.addOption("drain", "drain_only", false,
         "Drain the region server(Works only with hadoopctl). Default : false");
+    options.addOption("undrain", "undrain_only", false,
+        "Undrain the region server(Works only with hadoopctl). Requires drain log. Default: false");
+    options.addOption("location", "drain_log_location", true,
+        "Specifies file name where drain log should be written to and where it should be read from. " +
+            "Default: new temp file for drain, none for undrain");
+    options.addOption("readlog", "read_log_file", false,
+        "Reads the drain log file and prints out regions. User -location to set path to the log file. Default: false");
+
 
     if (args.length == 0) {
       HelpFormatter formatter = new HelpFormatter();
@@ -500,9 +550,19 @@ public class RollingRestart {
     int getOpFrequency = RollingRestart.DEFAULT_GETOP_FREQUENCY;
     int sleepIntervalBeforeRestart = RollingRestart.DEFAULT_SLEEP_BEFORE_RESTART_INTERVAL;
     boolean useHadoopCtl = true;
-    boolean drainAndStopOnly = false;
-    boolean drainOnly = false;
     int port = HConstants.DEFAULT_REGIONSERVER_PORT;
+    RollingRestartMode mode = null;
+
+    String drainLogFileName = cmd.getOptionValue("location", null);
+
+    if (cmd.hasOption("readlog")) {
+      if (drainLogFileName == null) {
+        LOG.error("Drain log file is not specified, use -location");
+        return;
+      }
+      printLogFileContent(drainLogFileName);
+      return;
+    }
 
     if (!cmd.hasOption("s")) {
       HelpFormatter formatter = new HelpFormatter();
@@ -524,8 +584,32 @@ public class RollingRestart {
       useHadoopCtl = false;
     }
 
+    if (cmd.hasOption("undrain")) {
+      if (drainLogFileName == null) {
+        LOG.error("Undrain requires drain log. Use -location /path/to/drain/log");
+        return;
+      }
+      mode = RollingRestartMode.UNDRAIN_ONLY;
+    }
+
     if (cmd.hasOption("o")) {
-      drainAndStopOnly = true;
+      if (mode != null) {
+        LOG.error("Multiple modes are specified: -o should not be used with -undrain or -drain");
+        return;
+      }
+      mode = RollingRestartMode.DRAIN_AND_STOP_ONLY;
+    }
+
+    if (cmd.hasOption("drain")) {
+      if (mode != null) {
+        LOG.error("Multiple modes are specified: -drain should not be used with -undrain or -o");
+        return;
+      }
+      mode = RollingRestartMode.DRAIN_ONLY;
+    }
+
+    if (mode == null) { //by default do full rolling restart
+      mode = RollingRestartMode.ALL;
     }
 
     if (cmd.hasOption("d")) {
@@ -544,18 +628,14 @@ public class RollingRestart {
       port = Integer.parseInt(cmd.getOptionValue("p"));
     }
 
-    if (cmd.hasOption("drain")) {
-      drainOnly = true;
-    }
 
-    RollingRestart rr = null;
+    RollingRestart rr;
     try {
       rr = new RollingRestart(serverName, regionDrainInterval,
           regionUndrainInterval, sleepIntervalAfterRestart,
           sleepIntervalBeforeRestart, getOpFrequency, useHadoopCtl, port);
     } catch (IOException e) {
-      e.printStackTrace();
-      LOG.error("Rolling restart failed for " + serverName);
+      LOG.error("Rolling restart failed for " + serverName, e);
       return;
     }
 
@@ -563,28 +643,40 @@ public class RollingRestart {
     Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.INFO);
 
     if (cmd.hasOption("c")) {
-      rr.clear(false);
+      rr.clear();
+      rr.removeFromBlackList();
       return;
     }
 
     try  {
       rr.setup();
-      rr.drainServer();
-      if (drainOnly) {
-        LOG.info("Drain completed for " + serverName);
-        return;
+
+      File drainedLogFile = (drainLogFileName != null)? new File(drainLogFileName) : null;
+      if (drainedLogFile == null) {
+        drainedLogFile = File.createTempFile("drained_regions_", ".bin");
       }
-      rr.restart(drainAndStopOnly);
-      if (!drainAndStopOnly) {
-        rr.undrainServer();
-        LOG.info("Rolling restart complete for " + serverName);
-      } else {
-        LOG.info("Drain complete for " + serverName);
+
+      switch (mode) {
+        case DRAIN_ONLY:
+          rr.drainServer(drainedLogFile);
+          break;
+        case UNDRAIN_ONLY:
+          rr.undrainServer(drainedLogFile);
+          break;
+        case DRAIN_AND_STOP_ONLY:
+          rr.drainServer(drainedLogFile);
+          rr.stop();
+          break;
+        case ALL:
+          rr.drainServer(drainedLogFile);
+          rr.restart();
+          rr.undrainServer(drainedLogFile);
+          drainedLogFile.deleteOnExit();
+          break;
       }
-      
+
     } catch (Exception e) {
-      e.printStackTrace();
-      LOG.error("Rolling restart failed for " + serverName + " at stage " + rr.currentState.name());
+      LOG.error("Rolling restart failed for " + serverName + " at stage " + rr.currentState.name(), e);
       switch (rr.currentState) {
         case SETUP:
           LOG.error("Cannot start rolling restart. Please retry");
@@ -604,7 +696,49 @@ public class RollingRestart {
          default:
        }
     } finally {
-      rr.clear(drainOnly | drainAndStopOnly);
+      rr.clear();
+      if (mode == RollingRestartMode.UNDRAIN_ONLY || mode == RollingRestartMode.ALL) {
+          rr.removeFromBlackList();
+      }
+    }
+  }
+
+  private static void printLogFileContent(String logFile) {
+    File file = new File(logFile);
+    if (!file.exists()) {
+      LOG.error("File " + logFile + " does not exist");
+      return;
+    }
+    HRegionInfo[] regions;
+    try {
+      regions = readDrainFile(file);
+    } catch (IOException e) {
+      LOG.error("Unable to read file " + logFile, e);
+      return;
+    }
+    StringBuilder message = new StringBuilder("Regions contained in the file: \n");
+    for (HRegionInfo region : regions) {
+      message.append(region.getRegionNameAsString()).append('\n');
+    }
+    LOG.info(message.toString());
+  }
+
+  private static HRegionInfo[] readDrainFile(File drainFile) throws IOException {
+    RandomAccessFile stream = new RandomAccessFile(drainFile, "r");
+    List<HRegionInfo> regions = new ArrayList<>();
+    while (stream.getFilePointer() != stream.length()) {
+      HRegionInfo regionInfo = new HRegionInfo();
+      regionInfo.readFields(stream);
+      regions.add(regionInfo);
     }
+    return regions.toArray(new HRegionInfo[regions.size()]);
+  }
+
+
+  static enum RollingRestartMode {
+    DRAIN_ONLY,
+    DRAIN_AND_STOP_ONLY,
+    UNDRAIN_ONLY,
+    ALL
   }
 }

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestDrainRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestDrainRegionServer.java?rev=1585805&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestDrainRegionServer.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestDrainRegionServer.java Tue Apr  8 18:18:37 2014
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2014 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerInfo;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.master.AssignmentPlan;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * This test runs drain and undrain on single region server from the cluster to test
+ * RollingRestart utility. Before testing it sets up a cluster of 5 region servers and then pick one.
+ *
+ * Test requires write access to file system and takes more than a minute to run.
+ */
+public class TestDrainRegionServer {
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final byte[] FAM_NAM = Bytes.toBytes("f");
+
+  private static final int REGION_SERVERS = 5;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().set("hbase.loadbalancer.impl",
+        "org.apache.hadoop.hbase.master.RegionManager$AssignmentLoadBalancer");
+    TEST_UTIL.startMiniCluster(REGION_SERVERS);
+  }
+
+  @Test
+  public void testDrainAndUndrain() throws Exception {
+
+    byte[] tableName = Bytes.toBytes("testBlacklistRegionServerWithoutTimeout");
+
+    HTable table = TEST_UTIL.createTable(tableName, FAM_NAM);
+    TEST_UTIL.createMultiRegions(table, FAM_NAM);
+
+    List<JVMClusterUtil.RegionServerThread> servers =
+        TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads();
+
+    AssignmentPlan ap = TEST_UTIL.getHBaseCluster().
+      getMaster().regionPlacement.getNewAssignmentPlan();
+    TEST_UTIL.getHBaseCluster().getMaster().regionPlacement.updateAssignmentPlan(ap);
+
+    // Wait for rebalance to to complete
+    Thread.sleep(60000);
+
+    HRegionServer server = null;
+
+    // Lets select a server which does not have META/ROOT
+    for (int i = 0; i < servers.size(); i++) {
+      server = servers.get(i).getRegionServer();
+      for (HRegion region : server.getOnlineRegions()) {
+        if (region.getRegionInfo().isMetaRegion() ||
+            region.getRegionInfo().isRootRegion() ||
+            region.getRegionNameAsString().contains(",,")) {
+          server = null;
+          break;
+        }
+      }
+      if (server != null) {
+        break;
+      }
+    }
+
+    if (server == null) {
+      fail("Unable to find any server");
+    }
+
+    RollingRestart rollingRestart = getRollingRestart(server);
+    File drainFile = File.createTempFile("TestDrainRegionServer_", ".bin");
+    drainFile.deleteOnExit();
+
+    int regionsCount = server.getOnlineRegions().size();
+
+    assertTrue("Server to be drained has to have more than 0 regions", regionsCount > 0);
+
+    rollingRestart.drainServer(drainFile);
+
+    assertEquals("Drained server expected to have 0 online regions", 0, server.getOnlineRegions().size());
+
+    rollingRestart.undrainServer(drainFile);
+
+    assertEquals("Undrained server expected to have same number of online regions as before",
+      regionsCount, server.getOnlineRegions().size());
+  }
+
+  private RollingRestart getRollingRestart(HRegionServer server) throws IOException {
+    HServerInfo serverInfo = server.getServerInfo();
+    String serverName = serverInfo.getHostname();
+    int port = serverInfo.getServerAddress().getPort();
+    int sleepIntervalAfterRestart = RollingRestart.DEFAULT_SLEEP_AFTER_RESTART_INTERVAL;
+    int regionDrainInterval = RollingRestart.DEFAULT_REGION_DRAIN_INTERVAL;
+    int regionUndrainInterval = RollingRestart.DEFAULT_REGION_UNDRAIN_INTERVAL;
+    int getOpFrequency = RollingRestart.DEFAULT_GETOP_FREQUENCY;
+    int sleepIntervalBeforeRestart = RollingRestart.DEFAULT_SLEEP_BEFORE_RESTART_INTERVAL;
+
+    RollingRestart rollingRestart = new RollingRestart(serverName, regionDrainInterval, regionUndrainInterval,
+        sleepIntervalAfterRestart, sleepIntervalBeforeRestart,
+        getOpFrequency, true, port, TEST_UTIL.getConfiguration());
+    rollingRestart.setup();
+    return rollingRestart;
+  }
+}