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