Repository: hbase Updated Branches: refs/heads/master 49056295e -> 451798cef
HBASE-12192 Remove EventHandlerListener Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/451798ce Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/451798ce Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/451798ce Branch: refs/heads/master Commit: 451798cefb008b32bd2c1f18962e9dded9a887eb Parents: 4905629 Author: Ryan Rawson <ryano...@gmail.com> Authored: Sat Oct 18 14:27:04 2014 -0700 Committer: Ryan Rawson <ryano...@gmail.com> Committed: Sat Oct 18 14:27:04 2014 -0700 ---------------------------------------------------------------------- .../hadoop/hbase/executor/EventHandler.java | 40 ----------------- .../hadoop/hbase/executor/ExecutorService.java | 46 ++------------------ .../hadoop/hbase/master/MasterRpcServices.java | 2 +- .../apache/hadoop/hbase/client/TestAdmin.java | 26 ----------- 4 files changed, 5 insertions(+), 109 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/451798ce/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java index bf1f251..cbc0e56 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java @@ -48,11 +48,6 @@ import org.htrace.TraceScope; * hbase executor, see ExecutorService, has a switch for passing * event type to executor. * <p> - * Event listeners can be installed and will be called pre- and post- process if - * this EventHandler is run in a Thread (its a Runnable so if its {@link #run()} - * method gets called). Implement - * {@link EventHandlerListener}s, and registering using - * {@link #setListener(EventHandlerListener)}. * @see ExecutorService */ @InterfaceAudience.Private @@ -70,31 +65,12 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> { // sequence id for this event private final long seqid; - // Listener to call pre- and post- processing. May be null. - private EventHandlerListener listener; - // Time to wait for events to happen, should be kept short protected int waitingTimeForEvents; private final Span parent; /** - * This interface provides pre- and post-process hooks for events. - */ - public interface EventHandlerListener { - /** - * Called before any event is processed - * @param event The event handler whose process method is about to be called. - */ - void beforeProcess(EventHandler event); - /** - * Called after any event is processed - * @param event The event handler whose process method is about to be called. - */ - void afterProcess(EventHandler event); - } - - /** * Default base class constructor. */ public EventHandler(Server server, EventType eventType) { @@ -124,9 +100,7 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> { public void run() { TraceScope chunk = Trace.startSpan(this.getClass().getSimpleName(), parent); try { - if (getListener() != null) getListener().beforeProcess(this); process(); - if (getListener() != null) getListener().afterProcess(this); } catch(Throwable t) { handleException(t); } finally { @@ -187,20 +161,6 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> { return (this.seqid < eh.seqid) ? -1 : 1; } - /** - * @return Current listener or null if none set. - */ - public synchronized EventHandlerListener getListener() { - return listener; - } - - /** - * @param listener Listener to call pre- and post- {@link #process()}. - */ - public synchronized void setListener(EventHandlerListener listener) { - this.listener = listener; - } - @Override public String toString() { return "Event #" + getSeqid() + http://git-wip-us.apache.org/repos/asf/hbase/blob/451798ce/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java index 42cca2b..410fb39 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java @@ -35,7 +35,6 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener; import org.apache.hadoop.hbase.monitoring.ThreadMonitoring; import com.google.common.collect.Lists; @@ -52,10 +51,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; * call {@link #shutdown()}. * * <p>In order to use the service created above, call - * {@link #submit(EventHandler)}. Register pre- and post- processing listeners - * by registering your implementation of {@link EventHandler.EventHandlerListener} - * with {@link #registerListener(EventType, EventHandler.EventHandlerListener)}. Be sure - * to deregister your listener when done via {@link #unregisterListener(EventType)}. + * {@link #submit(EventHandler)}. */ @InterfaceAudience.Private public class ExecutorService { @@ -65,10 +61,6 @@ public class ExecutorService { private final ConcurrentHashMap<String, Executor> executorMap = new ConcurrentHashMap<String, Executor>(); - // listeners that are called before and after an event is processed - private ConcurrentHashMap<EventType, EventHandlerListener> eventHandlerListeners = - new ConcurrentHashMap<EventType, EventHandlerListener>(); - // Name of the server hosting this executor service. private final String servername; @@ -91,7 +83,7 @@ public class ExecutorService { throw new RuntimeException("An executor service with the name " + name + " is already running!"); } - Executor hbes = new Executor(name, maxThreads, this.eventHandlerListeners); + Executor hbes = new Executor(name, maxThreads); if (this.executorMap.putIfAbsent(name, hbes) != null) { throw new RuntimeException("An executor service with the name " + name + " is already running (2)!"); @@ -130,7 +122,7 @@ public class ExecutorService { String name = type.getExecutorName(this.servername); if (isExecutorServiceRunning(name)) { LOG.debug("Executor service " + toString() + " already running on " + - this.servername); + this.servername); return; } startExecutorService(name, maxThreads); @@ -149,28 +141,6 @@ public class ExecutorService { } } - /** - * Subscribe to updates before and after processing instances of - * {@link EventType}. Currently only one listener per - * event type. - * @param type Type of event we're registering listener for - * @param listener The listener to run. - */ - public void registerListener(final EventType type, - final EventHandlerListener listener) { - this.eventHandlerListeners.put(type, listener); - } - - /** - * Stop receiving updates before and after processing instances of - * {@link EventType} - * @param type Type of event we're registering listener for - * @return The listener we removed or null if we did not remove it. - */ - public EventHandlerListener unregisterListener(final EventType type) { - return this.eventHandlerListeners.remove(type); - } - public Map<String, ExecutorStatus> getAllExecutorStatuses() { Map<String, ExecutorStatus> ret = Maps.newHashMap(); for (Map.Entry<String, Executor> e : executorMap.entrySet()) { @@ -190,15 +160,12 @@ public class ExecutorService { // work queue to use - unbounded queue final BlockingQueue<Runnable> q = new LinkedBlockingQueue<Runnable>(); private final String name; - private final Map<EventType, EventHandlerListener> eventHandlerListeners; private static final AtomicLong seqids = new AtomicLong(0); private final long id; - protected Executor(String name, int maxThreads, - final Map<EventType, EventHandlerListener> eventHandlerListeners) { + protected Executor(String name, int maxThreads) { this.id = seqids.incrementAndGet(); this.name = name; - this.eventHandlerListeners = eventHandlerListeners; // create the thread pool executor this.threadPoolExecutor = new TrackingThreadPoolExecutor( maxThreads, maxThreads, @@ -216,11 +183,6 @@ public class ExecutorService { void submit(final EventHandler event) { // If there is a listener for this type, make sure we call the before // and after process methods. - EventHandlerListener listener = - this.eventHandlerListeners.get(event.getEventType()); - if (listener != null) { - event.setListener(listener); - } this.threadPoolExecutor.execute(event); } http://git-wip-us.apache.org/repos/asf/hbase/blob/451798ce/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 00078fd..ff3d15f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -172,7 +172,7 @@ import com.google.protobuf.ServiceException; @InterfaceAudience.Private @SuppressWarnings("deprecation") public class MasterRpcServices extends RSRpcServices - implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface { + implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface { protected static final Log LOG = LogFactory.getLog(MasterRpcServices.class.getName()); private final HMaster master; http://git-wip-us.apache.org/repos/asf/hbase/blob/451798ce/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index aa9066a..edb1f6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -560,32 +560,6 @@ public class TestAdmin { "hbase.online.schema.update.enable", true); } - /** - * Listens for when an event is done in Master. - */ - static class DoneListener implements EventHandler.EventHandlerListener { - private final AtomicBoolean done; - - DoneListener(final AtomicBoolean done) { - super(); - this.done = done; - } - - @Override - public void afterProcess(EventHandler event) { - this.done.set(true); - synchronized (this.done) { - // Wake anyone waiting on this value to change. - this.done.notifyAll(); - } - } - - @Override - public void beforeProcess(EventHandler event) { - // continue - } - } - @SuppressWarnings("deprecation") protected void verifyRoundRobinDistribution(HTable ht, int expectedRegions) throws IOException { int numRS = ht.getConnection().getCurrentNrHRS();