You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 19:41:55 UTC
svn commit: r1181916 - in /hbase/branches/0.89/src:
main/java/org/apache/hadoop/hbase/regionserver/Store.java
test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
Author: nspiegelberg
Date: Tue Oct 11 17:41:55 2011
New Revision: 1181916
URL: http://svn.apache.org/viewvc?rev=1181916&view=rev
Log:
Don't schedule major compactions during peak time.
Summary:
Major compactions triggering at peak time cause a lot of trouble as
they load up the cluster quite a bit with a lot of IO. We should have logic in
place which prevents major compactions from triggering at peak time. This is an
attempt in that direction.
Test Plan: 1) Unit test the peak time logic.
Reviewed By: mbautin
Reviewers: mbautin, nspiegelberg
CC: hbase@lists, , mbautin
Revert Plan:
Tags:
- begin *PUBLIC* platform impact section -
Bugzilla: #
- end platform impact -
Differential Revision: 294514
Task ID: 639114
Modified:
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1181916&r1=1181915&r2=1181916&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Tue Oct 11 17:41:55 2011
@@ -22,8 +22,10 @@ package org.apache.hadoop.hbase.regionse
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
+import java.util.Calendar;
import java.util.Collection;
import java.util.Collections;
+import java.util.GregorianCalendar;
import java.util.List;
import java.util.NavigableSet;
import java.util.SortedSet;
@@ -112,6 +114,9 @@ public class Store implements HeapSize {
final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
private final String storeNameStr;
private final boolean inMemory;
+ private int peakStartHour;
+ private int peakEndHour;
+ private final static Calendar calendar = new GregorianCalendar();
/*
* List of store files inside this store. This is an immutable list that
@@ -206,6 +211,19 @@ public class Store implements HeapSize {
"hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
}
this.storefiles = sortAndClone(loadStoreFiles());
+ // Peak time is from [peakStartHour, peakEndHour). Valid numbers are [0, 23]
+ this.peakStartHour = conf.getInt("hbase.peak.start.hour", -1);
+ this.peakEndHour = conf.getInt("hbase.peak.end.hour", -1);
+ if (!isValidHour(this.peakStartHour) || !isValidHour(this.peakEndHour)) {
+ this.peakStartHour = this.peakEndHour = -1;
+ LOG.warn("Invalid start/end hour for peak hour : start = " +
+ this.peakStartHour + " end = " + this.peakEndHour +
+ ". Valid numbers are [0-23]");
+ }
+ }
+
+ private boolean isValidHour(int hour) {
+ return (hour >= 0 && hour <= 23);
}
public HColumnDescriptor getFamily() {
@@ -743,6 +761,17 @@ public class Store implements HeapSize {
return isMajorCompaction(candidates);
}
+ boolean isPeakTime(int currentHour) {
+ // Peak time checking is disabled just return false.
+ if (this.peakStartHour == this.peakEndHour) {
+ return false;
+ }
+ if (this.peakStartHour <= this.peakEndHour) {
+ return (currentHour >= this.peakStartHour && currentHour < this.peakEndHour);
+ }
+ return (currentHour >= this.peakStartHour || currentHour < this.peakEndHour);
+ }
+
/*
* @param filesToCompact Files to compact. Can be null.
* @return True if we should run a major compaction.
@@ -766,6 +795,12 @@ public class Store implements HeapSize {
" because one (major) compacted file only and elapsedTime " +
elapsedTime + "ms is < ttl=" + this.ttl);
}
+ } else if (isPeakTime(calendar.get(Calendar.HOUR_OF_DAY))) {
+ LOG.debug("Peak traffic time for HBase, not scheduling any major " +
+ "compactions. Peak hour period is : " + this.peakStartHour + " - " +
+ this.peakEndHour + " current hour is : " +
+ calendar.get(Calendar.HOUR_OF_DAY));
+ result = false;
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Major compaction triggered on store " + this.storeNameStr +
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java?rev=1181916&r1=1181915&r2=1181916&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java Tue Oct 11 17:41:55 2011
@@ -575,4 +575,30 @@ public class TestStore extends TestCase
result = HBaseTestingUtility.getFromStoreFile(store, get);
assertTrue(result.size()==0);
}
+
+ public void testIsPeakTime() throws IOException {
+ String methodName = "/testIsPeakTime";
+ final int[][] testcases = new int[][] { { 11, 15 }, { 23, 3 }, { 23, 0 },
+ { 0, 0 }, { 0, 23 } };
+ final int [][] tests = new int[][] {{ 10, 13, 16 }, { 0, 1, 4 },
+ { 22, 1, 23 }, { 1, 0, 2 }, { 0, 23, 16 }};
+ final boolean[][] answers = new boolean[][] {{false, true, false},
+ {true, true, false}, {false, false, true}, {false, false , false},
+ {true, false, true}};
+ assertEquals(testcases.length, tests.length);
+ assertEquals(tests.length, answers.length);
+ for (int i = 0; i < testcases.length; i++) {
+ int s = testcases[i][0];
+ int e = testcases[i][1];
+ Configuration conf = HBaseConfiguration.create();
+ conf.setInt("hbase.peak.start.hour", s);
+ conf.setInt("hbase.peak.end.hour", e);
+ assertEquals(tests[i].length, answers[i].length);
+ for (int j = 0; j < tests[i].length; j++) {
+ String method = methodName + i + "_" + j;
+ init(method, conf);
+ assertEquals(answers[i][j], this.store.isPeakTime(tests[i][j]));
+ }
+ }
+ }
}