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 */ 018package org.apache.hadoop.hbase.chaos.util; 019 020import java.io.Closeable; 021import java.io.IOException; 022import java.util.concurrent.ExecutorService; 023import java.util.concurrent.Executors; 024import java.util.concurrent.TimeUnit; 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.hbase.HBaseConfiguration; 027import org.apache.hadoop.hbase.IntegrationTestingUtility; 028import org.slf4j.Logger; 029import org.slf4j.LoggerFactory; 030 031import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 032import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 033 034/** 035 * This class can be used to control chaos monkeys life cycle. 036 */ 037public class Monkeys implements Closeable { 038 private static final Logger LOG = LoggerFactory.getLogger(Monkeys.class); 039 040 private final Configuration conf; 041 private final ChaosMonkeyRunner monkeyRunner; 042 private final Runnable runner; 043 private final ExecutorService executor; 044 045 public Monkeys() { 046 this(HBaseConfiguration.create()); 047 } 048 049 public Monkeys(Configuration conf) { 050 this.conf = Preconditions.checkNotNull(conf, "Should specify a configuration"); 051 this.monkeyRunner = new ChaosMonkeyRunner(); 052 this.runner = () -> { 053 try { 054 monkeyRunner.getAndStartMonkey(); 055 } catch (Exception e) { 056 LOG.error("Exception occurred when running chaos monkeys: ", e); 057 } 058 }; 059 this.executor = Executors.newSingleThreadExecutor( 060 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ChaosMonkey").build()); 061 IntegrationTestingUtility.setUseDistributedCluster(this.conf); 062 } 063 064 public void addResource(Configuration otherConf) { 065 conf.addResource(otherConf); 066 monkeyRunner.setConf(conf); 067 } 068 069 public void addResource(String otherConf) { 070 conf.addResource(otherConf); 071 monkeyRunner.setConf(conf); 072 } 073 074 public void startChaos() { 075 executor.execute(runner); 076 LOG.info("Chaos monkeys are running."); 077 } 078 079 public void stopChaos() { 080 monkeyRunner.stopRunner(); 081 LOG.info("Chaos monkeys are stopped."); 082 } 083 084 @Override 085 public void close() throws IOException { 086 executor.shutdown(); 087 try { 088 // wait 10 seconds. 089 executor.awaitTermination(10, TimeUnit.SECONDS); 090 } catch (InterruptedException e) { 091 Thread.currentThread().interrupt(); 092 LOG.warn("Interruption occurred while stopping chaos monkeys " + e); 093 } 094 } 095}