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.policies; 020 021import java.util.concurrent.ExecutionException; 022import java.util.concurrent.ExecutorService; 023import java.util.concurrent.Executors; 024import java.util.concurrent.Future; 025import org.apache.hadoop.hbase.chaos.actions.Action; 026import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; 027import org.apache.hadoop.hbase.util.Threads; 028import org.apache.hadoop.util.StringUtils; 029 030import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 031 032/** 033 * Chaos Monkey policy that will run two different actions at the same time. 034 * A random action from each array of actions will be chosen and then run in parallel. 035 */ 036public class TwoConcurrentActionPolicy extends PeriodicPolicy { 037 private final Action[] actionsOne; 038 private final Action[] actionsTwo; 039 private final ExecutorService executor; 040 041 public TwoConcurrentActionPolicy(long sleepTime, Action[] actionsOne, Action[] actionsTwo) { 042 super(sleepTime); 043 this.actionsOne = actionsOne; 044 this.actionsTwo = actionsTwo; 045 executor = Executors.newFixedThreadPool(2, 046 new ThreadFactoryBuilder().setNameFormat("TwoConcurrentAction-pool-%d").setDaemon(true) 047 .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); 048 } 049 050 @Override 051 protected void runOneIteration() { 052 Action actionOne = PolicyBasedChaosMonkey.selectRandomItem(actionsOne); 053 Action actionTwo = PolicyBasedChaosMonkey.selectRandomItem(actionsTwo); 054 055 Future fOne = executor.submit(new ActionRunner(actionOne)); 056 Future fTwo = executor.submit(new ActionRunner(actionTwo)); 057 058 try { 059 fOne.get(); 060 fTwo.get(); 061 } catch (InterruptedException e) { 062 LOG.warn("Exception occurred during performing action: " 063 + StringUtils.stringifyException(e)); 064 } catch (ExecutionException ex) { 065 LOG.warn("Exception occurred during performing action: " 066 + StringUtils.stringifyException(ex)); 067 } 068 } 069 070 @Override 071 public void init(PolicyContext context) throws Exception { 072 super.init(context); 073 for (Action a : actionsOne) { 074 a.init(context); 075 } 076 for (Action a : actionsTwo) { 077 a.init(context); 078 } 079 } 080 081 private static class ActionRunner implements Runnable { 082 083 private final Action action; 084 085 public ActionRunner(Action action) { 086 087 this.action = action; 088 } 089 090 @Override public void run() { 091 try { 092 action.perform(); 093 } catch (Exception ex) { 094 LOG.warn("Exception occurred during performing action: " 095 + StringUtils.stringifyException(ex)); 096 } 097 } 098 } 099}