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.errorhandling; 019 020import java.util.Timer; 021import java.util.TimerTask; 022 023import org.apache.yetus.audience.InterfaceAudience; 024import org.slf4j.Logger; 025import org.slf4j.LoggerFactory; 026import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 027 028/** 029 * Time a given process/operation and report a failure if the elapsed time exceeds the max allowed 030 * time. 031 * <p> 032 * The timer won't start tracking time until calling {@link #start()}. If {@link #complete()} or 033 * {@link #trigger()} is called before {@link #start()}, calls to {@link #start()} will fail. 034 */ 035@InterfaceAudience.Private 036public class TimeoutExceptionInjector { 037 038 private static final Logger LOG = LoggerFactory.getLogger(TimeoutExceptionInjector.class); 039 040 private final long maxTime; 041 private volatile boolean complete; 042 private final Timer timer; 043 private final TimerTask timerTask; 044 private long start = -1; 045 046 /** 047 * Create a generic timer for a task/process. 048 * @param listener listener to notify if the process times out 049 * @param maxTime max allowed running time for the process. Timer starts on calls to 050 * {@link #start()} 051 */ 052 public TimeoutExceptionInjector(final ForeignExceptionListener listener, final long maxTime) { 053 this.maxTime = maxTime; 054 timer = new Timer(); 055 timerTask = new TimerTask() { 056 @Override 057 public void run() { 058 // ensure we don't run this task multiple times 059 synchronized (this) { 060 // quick exit if we already marked the task complete 061 if (TimeoutExceptionInjector.this.complete) return; 062 // mark the task is run, to avoid repeats 063 TimeoutExceptionInjector.this.complete = true; 064 } 065 long end = EnvironmentEdgeManager.currentTime(); 066 TimeoutException tee = new TimeoutException( 067 "Timeout caused Foreign Exception", start, end, maxTime); 068 String source = "timer-" + timer; 069 listener.receive(new ForeignException(source, tee)); 070 } 071 }; 072 } 073 074 public long getMaxTime() { 075 return maxTime; 076 } 077 078 /** 079 * For all time forward, do not throw an error because the process has completed. 080 */ 081 public void complete() { 082 synchronized (this.timerTask) { 083 if (this.complete) { 084 LOG.warn("Timer already marked completed, ignoring!"); 085 return; 086 } 087 if (LOG.isDebugEnabled()) { 088 LOG.debug("Marking timer as complete - no error notifications will be received for " + 089 "this timer."); 090 } 091 this.complete = true; 092 } 093 this.timer.cancel(); 094 } 095 096 /** 097 * Start a timer to fail a process if it takes longer than the expected time to complete. 098 * <p> 099 * Non-blocking. 100 * @throws IllegalStateException if the timer has already been marked done via {@link #complete()} 101 * or {@link #trigger()} 102 */ 103 public synchronized void start() throws IllegalStateException { 104 if (this.start >= 0) { 105 LOG.warn("Timer already started, can't be started again. Ignoring second request."); 106 return; 107 } 108 LOG.debug("Scheduling process timer to run in: " + maxTime + " ms"); 109 timer.schedule(timerTask, maxTime); 110 this.start = EnvironmentEdgeManager.currentTime(); 111 } 112 113 /** 114 * Trigger the timer immediately. 115 * <p> 116 * Exposed for testing. 117 */ 118 public void trigger() { 119 synchronized (timerTask) { 120 if (this.complete) { 121 LOG.warn("Timer already completed, not triggering."); 122 return; 123 } 124 LOG.debug("Triggering timer immediately!"); 125 this.timer.cancel(); 126 this.timerTask.run(); 127 } 128 } 129}