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.regionserver.compactions; 019 020import org.apache.yetus.audience.InterfaceAudience; 021import org.slf4j.Logger; 022import org.slf4j.LoggerFactory; 023import org.apache.hadoop.conf.Configuration; 024 025@InterfaceAudience.Private 026public abstract class OffPeakHours { 027 private static final Logger LOG = LoggerFactory.getLogger(OffPeakHours.class); 028 029 public static final OffPeakHours DISABLED = new OffPeakHours() { 030 @Override public boolean isOffPeakHour() { return false; } 031 @Override public boolean isOffPeakHour(int targetHour) { return false; } 032 }; 033 034 public static OffPeakHours getInstance(Configuration conf) { 035 int startHour = conf.getInt(CompactionConfiguration.HBASE_HSTORE_OFFPEAK_START_HOUR, -1); 036 int endHour = conf.getInt(CompactionConfiguration.HBASE_HSTORE_OFFPEAK_END_HOUR, -1); 037 return getInstance(startHour, endHour); 038 } 039 040 /** 041 * @param startHour inclusive 042 * @param endHour exclusive 043 */ 044 public static OffPeakHours getInstance(int startHour, int endHour) { 045 if (startHour == -1 && endHour == -1) { 046 return DISABLED; 047 } 048 049 if (! isValidHour(startHour) || ! isValidHour(endHour)) { 050 if (LOG.isWarnEnabled()) { 051 LOG.warn("Ignoring invalid start/end hour for peak hour : start = " + 052 startHour + " end = " + endHour + 053 ". Valid numbers are [0-23]"); 054 } 055 return DISABLED; 056 } 057 058 if (startHour == endHour) { 059 return DISABLED; 060 } 061 062 return new OffPeakHoursImpl(startHour, endHour); 063 } 064 065 private static boolean isValidHour(int hour) { 066 return 0 <= hour && hour <= 23; 067 } 068 069 /** 070 * @return whether {@code targetHour} is off-peak hour 071 */ 072 public abstract boolean isOffPeakHour(int targetHour); 073 074 /** 075 * @return whether it is off-peak hour 076 */ 077 public abstract boolean isOffPeakHour(); 078 079 private static class OffPeakHoursImpl extends OffPeakHours { 080 final int startHour; 081 final int endHour; 082 083 /** 084 * @param startHour inclusive 085 * @param endHour exclusive 086 */ 087 OffPeakHoursImpl(int startHour, int endHour) { 088 this.startHour = startHour; 089 this.endHour = endHour; 090 } 091 092 @Override 093 public boolean isOffPeakHour() { 094 return isOffPeakHour(CurrentHourProvider.getCurrentHour()); 095 } 096 097 @Override 098 public boolean isOffPeakHour(int targetHour) { 099 if (startHour <= endHour) { 100 return startHour <= targetHour && targetHour < endHour; 101 } 102 return targetHour < endHour || startHour <= targetHour; 103 } 104 } 105}