001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.chaos.actions;
020
021import java.io.IOException;
022import java.util.ArrayList;
023import java.util.Collection;
024import java.util.HashSet;
025import java.util.LinkedList;
026import java.util.List;
027import java.util.Map;
028import java.util.Properties;
029import java.util.Set;
030import java.util.function.BiConsumer;
031import java.util.function.Consumer;
032import org.apache.commons.lang3.RandomUtils;
033import org.apache.hadoop.conf.Configuration;
034import org.apache.hadoop.hbase.ClusterMetrics;
035import org.apache.hadoop.hbase.HBaseCluster;
036import org.apache.hadoop.hbase.HBaseTestingUtility;
037import org.apache.hadoop.hbase.HRegionInfo;
038import org.apache.hadoop.hbase.IntegrationTestBase;
039import org.apache.hadoop.hbase.IntegrationTestingUtility;
040import org.apache.hadoop.hbase.MiniHBaseCluster;
041import org.apache.hadoop.hbase.ServerMetrics;
042import org.apache.hadoop.hbase.ServerName;
043import org.apache.hadoop.hbase.TableName;
044import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
045import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
046import org.apache.hadoop.hbase.client.Admin;
047import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
048import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
049import org.apache.hadoop.hbase.client.TableDescriptor;
050import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
051import org.apache.hadoop.hbase.util.Bytes;
052import org.slf4j.Logger;
053
054/**
055 * A (possibly mischievous) action that the ChaosMonkey can perform.
056 */
057public abstract class Action {
058
059  public static final String KILL_MASTER_TIMEOUT_KEY =
060    "hbase.chaosmonkey.action.killmastertimeout";
061  public static final String START_MASTER_TIMEOUT_KEY =
062    "hbase.chaosmonkey.action.startmastertimeout";
063  public static final String KILL_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.killrstimeout";
064  public static final String START_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.startrstimeout";
065  public static final String KILL_ZK_NODE_TIMEOUT_KEY =
066    "hbase.chaosmonkey.action.killzknodetimeout";
067  public static final String START_ZK_NODE_TIMEOUT_KEY =
068    "hbase.chaosmonkey.action.startzknodetimeout";
069  public static final String KILL_DATANODE_TIMEOUT_KEY =
070    "hbase.chaosmonkey.action.killdatanodetimeout";
071  public static final String START_DATANODE_TIMEOUT_KEY =
072    "hbase.chaosmonkey.action.startdatanodetimeout";
073  public static final String KILL_NAMENODE_TIMEOUT_KEY =
074      "hbase.chaosmonkey.action.killnamenodetimeout";
075  public static final String START_NAMENODE_TIMEOUT_KEY =
076      "hbase.chaosmonkey.action.startnamenodetimeout";
077
078  protected static final long KILL_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
079  protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
080  protected static final long KILL_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
081  protected static final long START_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
082  protected static final long KILL_ZK_NODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
083  protected static final long START_ZK_NODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
084  protected static final long KILL_DATANODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
085  protected static final long START_DATANODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
086  protected static final long KILL_NAMENODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
087  protected static final long START_NAMENODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
088
089  protected ActionContext context;
090  protected HBaseCluster cluster;
091  protected ClusterMetrics initialStatus;
092  protected ServerName[] initialServers;
093  protected Properties monkeyProps;
094
095  protected long killMasterTimeout;
096  protected long startMasterTimeout;
097  protected long killRsTimeout;
098  protected long startRsTimeout;
099  protected long killZkNodeTimeout;
100  protected long startZkNodeTimeout;
101  protected long killDataNodeTimeout;
102  protected long startDataNodeTimeout;
103  protected long killNameNodeTimeout;
104  protected long startNameNodeTimeout;
105  protected boolean skipMetaRS;
106
107  /**
108   * Retrieve the instance's {@link Logger}, for use throughout the class hierarchy.
109   */
110  protected abstract Logger getLogger();
111
112  public void init(ActionContext context) throws IOException {
113    this.context = context;
114    cluster = context.getHBaseCluster();
115    initialStatus = cluster.getInitialClusterMetrics();
116    Collection<ServerName> regionServers = initialStatus.getLiveServerMetrics().keySet();
117    initialServers = regionServers.toArray(new ServerName[0]);
118
119    monkeyProps = context.getMonkeyProps();
120    if (monkeyProps == null){
121      monkeyProps = new Properties();
122      IntegrationTestBase.loadMonkeyProperties(monkeyProps, cluster.getConf());
123    }
124
125    killMasterTimeout = Long.parseLong(monkeyProps.getProperty(
126      KILL_MASTER_TIMEOUT_KEY, KILL_MASTER_TIMEOUT_DEFAULT + ""));
127    startMasterTimeout = Long.parseLong(monkeyProps.getProperty(START_MASTER_TIMEOUT_KEY,
128      START_MASTER_TIMEOUT_DEFAULT + ""));
129    killRsTimeout = Long.parseLong(monkeyProps.getProperty(KILL_RS_TIMEOUT_KEY,
130      KILL_RS_TIMEOUT_DEFAULT + ""));
131    startRsTimeout = Long.parseLong(monkeyProps.getProperty(START_RS_TIMEOUT_KEY,
132      START_RS_TIMEOUT_DEFAULT+ ""));
133    killZkNodeTimeout = Long.parseLong(monkeyProps.getProperty(KILL_ZK_NODE_TIMEOUT_KEY,
134      KILL_ZK_NODE_TIMEOUT_DEFAULT + ""));
135    startZkNodeTimeout = Long.parseLong(monkeyProps.getProperty(START_ZK_NODE_TIMEOUT_KEY,
136      START_ZK_NODE_TIMEOUT_DEFAULT + ""));
137    killDataNodeTimeout = Long.parseLong(monkeyProps.getProperty(KILL_DATANODE_TIMEOUT_KEY,
138      KILL_DATANODE_TIMEOUT_DEFAULT + ""));
139    startDataNodeTimeout = Long.parseLong(monkeyProps.getProperty(START_DATANODE_TIMEOUT_KEY,
140      START_DATANODE_TIMEOUT_DEFAULT + ""));
141    killNameNodeTimeout = Long.parseLong(monkeyProps.getProperty(KILL_NAMENODE_TIMEOUT_KEY,
142      KILL_NAMENODE_TIMEOUT_DEFAULT + ""));
143    startNameNodeTimeout = Long.parseLong(monkeyProps.getProperty(START_NAMENODE_TIMEOUT_KEY,
144      START_NAMENODE_TIMEOUT_DEFAULT + ""));
145    skipMetaRS = Boolean.parseBoolean(monkeyProps.getProperty(MonkeyConstants.SKIP_META_RS,
146      MonkeyConstants.DEFAULT_SKIP_META_RS + ""));
147  }
148
149  public void perform() throws Exception { }
150
151  /** Returns current region servers - active master */
152  protected ServerName[] getCurrentServers() throws IOException {
153    ClusterMetrics clusterStatus = cluster.getClusterMetrics();
154    Collection<ServerName> regionServers = clusterStatus.getLiveServerMetrics().keySet();
155    int count = regionServers.size();
156    if (count <= 0) {
157      return new ServerName [] {};
158    }
159    ServerName master = clusterStatus.getMasterName();
160    Set<ServerName> masters = new HashSet<>();
161    masters.add(master);
162    masters.addAll(clusterStatus.getBackupMasterNames());
163    ArrayList<ServerName> tmp = new ArrayList<>(count);
164    tmp.addAll(regionServers);
165    tmp.removeAll(masters);
166
167    if(skipMetaRS){
168      ServerName metaServer = cluster.getServerHoldingMeta();
169      tmp.remove(metaServer);
170    }
171
172    return tmp.toArray(new ServerName[0]);
173  }
174
175  protected void killMaster(ServerName server) throws IOException {
176    getLogger().info("Killing master {}", server);
177    cluster.killMaster(server);
178    cluster.waitForMasterToStop(server, killMasterTimeout);
179    getLogger().info("Killed master " + server);
180  }
181
182  protected void startMaster(ServerName server) throws IOException {
183    getLogger().info("Starting master {}", server.getHostname());
184    cluster.startMaster(server.getHostname(), server.getPort());
185    cluster.waitForActiveAndReadyMaster(startMasterTimeout);
186    getLogger().info("Started master " + server.getHostname());
187  }
188
189  protected void stopRs(ServerName server) throws IOException {
190    getLogger().info("Stopping regionserver {}", server);
191    cluster.stopRegionServer(server);
192    cluster.waitForRegionServerToStop(server, killRsTimeout);
193    getLogger().info("Stopping regionserver {}. Reported num of rs:{}", server,
194      cluster.getClusterMetrics().getLiveServerMetrics().size());
195  }
196
197  protected void suspendRs(ServerName server) throws IOException {
198    getLogger().info("Suspending regionserver {}", server);
199    cluster.suspendRegionServer(server);
200    if(!(cluster instanceof MiniHBaseCluster)){
201      cluster.waitForRegionServerToStop(server, killRsTimeout);
202    }
203    getLogger().info("Suspending regionserver {}. Reported num of rs:{}", server,
204      cluster.getClusterMetrics().getLiveServerMetrics().size());
205  }
206
207  protected void resumeRs(ServerName server) throws IOException {
208    getLogger().info("Resuming regionserver {}", server);
209    cluster.resumeRegionServer(server);
210    if(!(cluster instanceof MiniHBaseCluster)){
211      cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
212    }
213    getLogger().info("Resuming regionserver {}. Reported num of rs:{}", server,
214      cluster.getClusterMetrics().getLiveServerMetrics().size());
215  }
216
217  protected void killRs(ServerName server) throws IOException {
218    getLogger().info("Killing regionserver {}", server);
219    cluster.killRegionServer(server);
220    cluster.waitForRegionServerToStop(server, killRsTimeout);
221    getLogger().info("Killed regionserver {}. Reported num of rs:{}", server,
222      cluster.getClusterMetrics().getLiveServerMetrics().size());
223  }
224
225  protected void startRs(ServerName server) throws IOException {
226    getLogger().info("Starting regionserver {}", server.getAddress());
227    cluster.startRegionServer(server.getHostname(), server.getPort());
228    cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
229    getLogger().info("Started regionserver {}. Reported num of rs:{}", server.getAddress(),
230      cluster.getClusterMetrics().getLiveServerMetrics().size());
231  }
232
233  protected void killZKNode(ServerName server) throws IOException {
234    getLogger().info("Killing zookeeper node {}", server);
235    cluster.killZkNode(server);
236    cluster.waitForZkNodeToStop(server, killZkNodeTimeout);
237    getLogger().info("Killed zookeeper node {}. Reported num of rs:{}", server,
238      cluster.getClusterMetrics().getLiveServerMetrics().size());
239  }
240
241  protected void startZKNode(ServerName server) throws IOException {
242    getLogger().info("Starting zookeeper node {}", server.getHostname());
243    cluster.startZkNode(server.getHostname(), server.getPort());
244    cluster.waitForZkNodeToStart(server, startZkNodeTimeout);
245    getLogger().info("Started zookeeper node {}", server);
246  }
247
248  protected void killDataNode(ServerName server) throws IOException {
249    getLogger().info("Killing datanode {}", server);
250    cluster.killDataNode(server);
251    cluster.waitForDataNodeToStop(server, killDataNodeTimeout);
252    getLogger().info("Killed datanode {}. Reported num of rs:{}", server,
253      cluster.getClusterMetrics().getLiveServerMetrics().size());
254  }
255
256  protected void startDataNode(ServerName server) throws IOException {
257    getLogger().info("Starting datanode {}", server.getHostname());
258    cluster.startDataNode(server);
259    cluster.waitForDataNodeToStart(server, startDataNodeTimeout);
260    getLogger().info("Started datanode {}", server);
261  }
262
263  protected void killNameNode(ServerName server) throws IOException {
264    getLogger().info("Killing namenode :-{}", server.getHostname());
265    cluster.killNameNode(server);
266    cluster.waitForNameNodeToStop(server, killNameNodeTimeout);
267    getLogger().info("Killed namenode:{}. Reported num of rs:{}", server,
268        cluster.getClusterMetrics().getLiveServerMetrics().size());
269  }
270
271  protected void startNameNode(ServerName server) throws IOException {
272    getLogger().info("Starting Namenode :-{}", server.getHostname());
273    cluster.startNameNode(server);
274    cluster.waitForNameNodeToStart(server, startNameNodeTimeout);
275    getLogger().info("Started namenode:{}", server);
276  }
277  protected void unbalanceRegions(ClusterMetrics clusterStatus,
278      List<ServerName> fromServers, List<ServerName> toServers,
279      double fractionOfRegions) throws Exception {
280    List<byte[]> victimRegions = new LinkedList<>();
281    for (Map.Entry<ServerName, ServerMetrics> entry
282      : clusterStatus.getLiveServerMetrics().entrySet()) {
283      ServerName sn = entry.getKey();
284      ServerMetrics serverLoad = entry.getValue();
285      // Ugh.
286      List<byte[]> regions = new LinkedList<>(serverLoad.getRegionMetrics().keySet());
287      int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
288      getLogger().debug("Removing {} regions from {}", victimRegionCount, sn);
289      for (int i = 0; i < victimRegionCount; ++i) {
290        int victimIx = RandomUtils.nextInt(0, regions.size());
291        String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));
292        victimRegions.add(Bytes.toBytes(regionId));
293      }
294    }
295
296    getLogger().info("Moving {} regions from {} servers to {} different servers",
297      victimRegions.size(), fromServers.size(), toServers.size());
298    Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
299    for (byte[] victimRegion : victimRegions) {
300      // Don't keep moving regions if we're
301      // trying to stop the monkey.
302      if (context.isStopping()) {
303        break;
304      }
305      int targetIx = RandomUtils.nextInt(0, toServers.size());
306      admin.move(victimRegion, toServers.get(targetIx));
307    }
308  }
309
310  protected void forceBalancer() throws Exception {
311    Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
312    boolean result = false;
313    try {
314      result = admin.balancer();
315    } catch (Exception e) {
316      getLogger().warn("Got exception while doing balance ", e);
317    }
318    if (!result) {
319      getLogger().error("Balancer didn't succeed");
320    }
321  }
322
323  protected void setBalancer(boolean onOrOff, boolean synchronous) throws Exception {
324    Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
325    try {
326      admin.balancerSwitch(onOrOff, synchronous);
327    } catch (Exception e) {
328      getLogger().warn("Got exception while switching balance ", e);
329    }
330  }
331
332  public Configuration getConf() {
333    return cluster.getConf();
334  }
335
336  /**
337   * Apply a transform to all columns in a given table. If there are no columns in a table
338   * or if the context is stopping does nothing.
339   * @param tableName the table to modify
340   * @param transform the modification to perform. Callers will have the
341   *                  column name as a string and a column family builder available to them
342   */
343  protected void modifyAllTableColumns(TableName tableName,
344    BiConsumer<String, ColumnFamilyDescriptorBuilder> transform) throws IOException {
345    HBaseTestingUtility util = this.context.getHBaseIntegrationTestingUtility();
346    Admin admin = util.getAdmin();
347
348    TableDescriptor tableDescriptor = admin.getDescriptor(tableName);
349    ColumnFamilyDescriptor[] columnDescriptors = tableDescriptor.getColumnFamilies();
350
351    if (columnDescriptors == null || columnDescriptors.length == 0) {
352      return;
353    }
354
355    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor);
356    for (ColumnFamilyDescriptor descriptor : columnDescriptors) {
357      ColumnFamilyDescriptorBuilder cfd = ColumnFamilyDescriptorBuilder.newBuilder(descriptor);
358      transform.accept(descriptor.getNameAsString(), cfd);
359      builder.modifyColumnFamily(cfd.build());
360    }
361
362    // Don't try the modify if we're stopping
363    if (this.context.isStopping()) {
364      return;
365    }
366    admin.modifyTable(builder.build());
367  }
368
369  /**
370   * Apply a transform to all columns in a given table.
371   * If there are no columns in a table or if the context is stopping does nothing.
372   * @param tableName the table to modify
373   * @param transform the modification to perform on each column family descriptor builder
374   */
375  protected void modifyAllTableColumns(TableName tableName,
376    Consumer<ColumnFamilyDescriptorBuilder> transform) throws IOException {
377    modifyAllTableColumns(tableName, (name, cfd) -> transform.accept(cfd));
378  }
379
380  /**
381   * Context for Action's
382   */
383  public static class ActionContext {
384    private IntegrationTestingUtility util;
385    private Properties monkeyProps = null;
386
387    public ActionContext(IntegrationTestingUtility util) {
388      this.util = util;
389    }
390
391    public ActionContext(Properties monkeyProps, IntegrationTestingUtility util) {
392      this.util = util;
393      this.monkeyProps = monkeyProps;
394    }
395
396    public Properties getMonkeyProps(){
397      return monkeyProps;
398    }
399
400    public IntegrationTestingUtility getHBaseIntegrationTestingUtility() {
401      return util;
402    }
403
404    public HBaseCluster getHBaseCluster() {
405      return util.getHBaseClusterInterface();
406    }
407
408    public boolean isStopping() {
409      return false;
410    }
411  }
412}