View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.chaos.actions;
20  
21  import java.io.IOException;
22  import java.util.Collection;
23  import java.util.LinkedList;
24  import java.util.List;
25  
26  import org.apache.commons.lang.math.RandomUtils;
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.hbase.ClusterStatus;
31  import org.apache.hadoop.hbase.HBaseCluster;
32  import org.apache.hadoop.hbase.HRegionInfo;
33  import org.apache.hadoop.hbase.IntegrationTestingUtility;
34  import org.apache.hadoop.hbase.ServerLoad;
35  import org.apache.hadoop.hbase.ServerName;
36  import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
37  import org.apache.hadoop.hbase.client.HBaseAdmin;
38  import org.apache.hadoop.hbase.util.Bytes;
39  
40  /**
41   * A (possibly mischievous) action that the ChaosMonkey can perform.
42   */
43  public class Action {
44  
45    public static final String KILL_MASTER_TIMEOUT_KEY =
46      "hbase.chaosmonkey.action.killmastertimeout";
47    public static final String START_MASTER_TIMEOUT_KEY =
48      "hbase.chaosmonkey.action.startmastertimeout";
49    public static final String KILL_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.killrstimeout";
50    public static final String START_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.startrstimeout";
51    public static final String KILL_ZK_NODE_TIMEOUT_KEY =
52      "hbase.chaosmonkey.action.killzknodetimeout";
53    public static final String START_ZK_NODE_TIMEOUT_KEY =
54      "hbase.chaosmonkey.action.startzknodetimeout";
55    public static final String KILL_DATANODE_TIMEOUT_KEY =
56      "hbase.chaosmonkey.action.killdatanodetimeout";
57    public static final String START_DATANODE_TIMEOUT_KEY =
58      "hbase.chaosmonkey.action.startdatanodetimeout";
59  
60    protected static Log LOG = LogFactory.getLog(Action.class);
61  
62    protected static final long KILL_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
63    protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
64    protected static final long KILL_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
65    protected static final long START_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
66    protected static final long KILL_ZK_NODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
67    protected static final long START_ZK_NODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
68    protected static final long KILL_DATANODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
69    protected static final long START_DATANODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
70  
71    protected ActionContext context;
72    protected HBaseCluster cluster;
73    protected ClusterStatus initialStatus;
74    protected ServerName[] initialServers;
75  
76    protected long killMasterTimeout;
77    protected long startMasterTimeout;
78    protected long killRsTimeout;
79    protected long startRsTimeout;
80    protected long killZkNodeTimeout;
81    protected long startZkNodeTimeout;
82    protected long killDataNodeTimeout;
83    protected long startDataNodeTimeout;
84  
85    public void init(ActionContext context) throws IOException {
86      this.context = context;
87      cluster = context.getHBaseCluster();
88      initialStatus = cluster.getInitialClusterStatus();
89      Collection<ServerName> regionServers = initialStatus.getServers();
90      initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
91  
92      killMasterTimeout = cluster.getConf().getLong(KILL_MASTER_TIMEOUT_KEY,
93        KILL_MASTER_TIMEOUT_DEFAULT);
94      startMasterTimeout = cluster.getConf().getLong(START_MASTER_TIMEOUT_KEY,
95        START_MASTER_TIMEOUT_DEFAULT);
96      killRsTimeout = cluster.getConf().getLong(KILL_RS_TIMEOUT_KEY, KILL_RS_TIMEOUT_DEFAULT);
97      startRsTimeout = cluster.getConf().getLong(START_RS_TIMEOUT_KEY, START_RS_TIMEOUT_DEFAULT);
98      killZkNodeTimeout = cluster.getConf().getLong(KILL_ZK_NODE_TIMEOUT_KEY,
99        KILL_ZK_NODE_TIMEOUT_DEFAULT);
100     startZkNodeTimeout = cluster.getConf().getLong(START_ZK_NODE_TIMEOUT_KEY,
101       START_ZK_NODE_TIMEOUT_DEFAULT);
102     killDataNodeTimeout = cluster.getConf().getLong(KILL_DATANODE_TIMEOUT_KEY,
103       KILL_DATANODE_TIMEOUT_DEFAULT);
104     startDataNodeTimeout = cluster.getConf().getLong(START_DATANODE_TIMEOUT_KEY,
105       START_DATANODE_TIMEOUT_DEFAULT);
106   }
107 
108   public void perform() throws Exception { }
109 
110   /** Returns current region servers */
111   protected ServerName[] getCurrentServers() throws IOException {
112     Collection<ServerName> regionServers = cluster.getClusterStatus().getServers();
113     if (regionServers == null || regionServers.size() <= 0) return new ServerName [] {};
114     return regionServers.toArray(new ServerName[regionServers.size()]);
115   }
116 
117   protected void killMaster(ServerName server) throws IOException {
118     LOG.info("Killing master:" + server);
119     cluster.killMaster(server);
120     cluster.waitForMasterToStop(server, killMasterTimeout);
121     LOG.info("Killed master server:" + server);
122   }
123 
124   protected void startMaster(ServerName server) throws IOException {
125     LOG.info("Starting master:" + server.getHostname());
126     cluster.startMaster(server.getHostname(), server.getPort());
127     cluster.waitForActiveAndReadyMaster(startMasterTimeout);
128     LOG.info("Started master: " + server);
129   }
130 
131   protected void killRs(ServerName server) throws IOException {
132     LOG.info("Killing region server:" + server);
133     cluster.killRegionServer(server);
134     cluster.waitForRegionServerToStop(server, killRsTimeout);
135     LOG.info("Killed region server:" + server + ". Reported num of rs:"
136         + cluster.getClusterStatus().getServersSize());
137   }
138 
139   protected void startRs(ServerName server) throws IOException {
140     LOG.info("Starting region server:" + server.getHostname());
141     cluster.startRegionServer(server.getHostname(), server.getPort());
142     cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
143     LOG.info("Started region server:" + server + ". Reported num of rs:"
144       + cluster.getClusterStatus().getServersSize());
145   }
146 
147   protected void killZKNode(ServerName server) throws IOException {
148     LOG.info("Killing zookeeper node:" + server);
149     cluster.killZkNode(server);
150     cluster.waitForZkNodeToStop(server, killZkNodeTimeout);
151     LOG.info("Killed zookeeper node:" + server + ". Reported num of rs:"
152       + cluster.getClusterStatus().getServersSize());
153   }
154 
155   protected void startZKNode(ServerName server) throws IOException {
156     LOG.info("Starting zookeeper node:" + server.getHostname());
157     cluster.startZkNode(server.getHostname(), server.getPort());
158     cluster.waitForZkNodeToStart(server, startZkNodeTimeout);
159     LOG.info("Started zookeeper node:" + server);
160   }
161 
162   protected void killDataNode(ServerName server) throws IOException {
163     LOG.info("Killing datanode:" + server);
164     cluster.killDataNode(server);
165     cluster.waitForDataNodeToStop(server, killDataNodeTimeout);
166     LOG.info("Killed datanode:" + server + ". Reported num of rs:"
167       + cluster.getClusterStatus().getServersSize());
168   }
169 
170   protected void startDataNode(ServerName server) throws IOException {
171     LOG.info("Starting datanode:" + server.getHostname());
172     cluster.startDataNode(server);
173     cluster.waitForDataNodeToStart(server, startDataNodeTimeout);
174     LOG.info("Started datanode:" + server);
175   }
176 
177   protected void unbalanceRegions(ClusterStatus clusterStatus,
178       List<ServerName> fromServers, List<ServerName> toServers,
179       double fractionOfRegions) throws Exception {
180     List<byte[]> victimRegions = new LinkedList<byte[]>();
181     for (ServerName server : fromServers) {
182       ServerLoad serverLoad = clusterStatus.getLoad(server);
183       // Ugh.
184       List<byte[]> regions = new LinkedList<byte[]>(serverLoad.getRegionsLoad().keySet());
185       int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
186       LOG.debug("Removing " + victimRegionCount + " regions from " + server.getServerName());
187       for (int i = 0; i < victimRegionCount; ++i) {
188         int victimIx = RandomUtils.nextInt(regions.size());
189         String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));
190         victimRegions.add(Bytes.toBytes(regionId));
191       }
192     }
193 
194     LOG.info("Moving " + victimRegions.size() + " regions from " + fromServers.size()
195         + " servers to " + toServers.size() + " different servers");
196     HBaseAdmin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
197     for (byte[] victimRegion : victimRegions) {
198       int targetIx = RandomUtils.nextInt(toServers.size());
199       admin.move(victimRegion, Bytes.toBytes(toServers.get(targetIx).getServerName()));
200     }
201   }
202 
203   protected void forceBalancer() throws Exception {
204     HBaseAdmin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
205     boolean result = false;
206     try {
207       result = admin.balancer();
208     } catch (Exception e) {
209       LOG.warn("Got exception while doing balance ", e);
210     }
211     if (!result) {
212       LOG.error("Balancer didn't succeed");
213     }
214   }
215 
216   public Configuration getConf() {
217     return cluster.getConf();
218   }
219 
220   /**
221    * Context for Action's
222    */
223   public static class ActionContext {
224     private IntegrationTestingUtility util;
225 
226     public ActionContext(IntegrationTestingUtility util) {
227       this.util = util;
228     }
229 
230     public IntegrationTestingUtility getHBaseIntegrationTestingUtility() {
231       return util;
232     }
233 
234     public HBaseCluster getHBaseCluster() {
235       return util.getHBaseClusterInterface();
236     }
237   }
238 }