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