HBASE-8638 add logging to compaction policy

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1487948 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
sershe 2013-05-30 17:55:04 +00:00
parent c920eb3c4c
commit e7ac644071
3 changed files with 35 additions and 6 deletions

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -395,7 +397,11 @@ public class CompactSplitThread implements CompactionRequestor {
} }
} }
} catch (IOException ex) { } catch (IOException ex) {
LOG.error("Compaction failed " + this, RemoteExceptionHandler.checkIOException(ex)); IOException remoteEx = RemoteExceptionHandler.checkIOException(ex);
LOG.error("Compaction failed " + this, remoteEx);
if (remoteEx != ex) {
LOG.info("Compaction failed at original callstack: " + formatStackTrace(ex));
}
server.checkFileSystem(); server.checkFileSystem();
} catch (Exception ex) { } catch (Exception ex) {
LOG.error("Compaction failed " + this, ex); LOG.error("Compaction failed " + this, ex);
@ -406,6 +412,14 @@ public class CompactSplitThread implements CompactionRequestor {
this.compaction.getRequest().afterExecute(); this.compaction.getRequest().afterExecute();
} }
private String formatStackTrace(Exception ex) {
StringWriter sw = new StringWriter();
PrintWriter pw = new PrintWriter(sw);
ex.printStackTrace(pw);
pw.flush();
return sw.toString();
}
@Override @Override
public int compareTo(CompactionRunner o) { public int compareTo(CompactionRunner o) {
// Only compare the underlying request, for queue sorting purposes. // Only compare the underlying request, for queue sorting purposes.

View File

@ -23,6 +23,8 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
@ -36,7 +38,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy { public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
private static final Log LOG = LogFactory.getLog(ExploringCompactionPolicy.class);
/** /**
* Constructor for ExploringCompactionPolicy. * Constructor for ExploringCompactionPolicy.
@ -57,6 +59,7 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
long bestSize = 0; long bestSize = 0;
long smallestSize = Long.MAX_VALUE; long smallestSize = Long.MAX_VALUE;
int opts = 0, optsInRatio = 0, bestStart = -1; // for debug logging
// Consider every starting place. // Consider every starting place.
for (int start = 0; start < candidates.size(); start++) { for (int start = 0; start < candidates.size(); start++) {
// Consider every different sub list permutation in between start and end with min files. // Consider every different sub list permutation in between start and end with min files.
@ -83,26 +86,34 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
smallestSize = size; smallestSize = size;
} }
if (size > comConf.getMaxCompactSize()) {
continue;
}
++opts;
if (size >= comConf.getMinCompactSize() if (size >= comConf.getMinCompactSize()
&& !filesInRatio(potentialMatchFiles, mayUseOffPeak)) { && !filesInRatio(potentialMatchFiles, mayUseOffPeak)) {
continue; continue;
} }
if (size > comConf.getMaxCompactSize()) { ++optsInRatio;
continue;
}
// Keep if this gets rid of more files. Or the same number of files for less io. // Keep if this gets rid of more files. Or the same number of files for less io.
if (potentialMatchFiles.size() > bestSelection.size() if (potentialMatchFiles.size() > bestSelection.size()
|| (potentialMatchFiles.size() == bestSelection.size() && size < bestSize)) { || (potentialMatchFiles.size() == bestSelection.size() && size < bestSize)) {
bestSelection = potentialMatchFiles; bestSelection = potentialMatchFiles;
bestSize = size; bestSize = size;
bestStart = start;
} }
} }
} }
if (bestSelection.size() == 0 && mightBeStuck) { if (bestSelection.size() == 0 && mightBeStuck) {
LOG.debug("Exploring compaction algorithm has selected " + smallest.size()
+ " files of size "+ smallestSize + " because the store might be stuck");
return new ArrayList<StoreFile>(smallest); return new ArrayList<StoreFile>(smallest);
} }
LOG.debug("Exploring compaction algorithm has selected " + bestSelection.size()
+ " files of size " + bestSize + " starting at candidate #" + bestStart +
" after considering " + opts + " permutations with " + optsInRatio + " in ratio");
return new ArrayList<StoreFile>(bestSelection); return new ArrayList<StoreFile>(bestSelection);
} }

View File

@ -89,6 +89,10 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
boolean mayBeStuck = (candidateFiles.size() - filesCompacting.size() + futureFiles) boolean mayBeStuck = (candidateFiles.size() - filesCompacting.size() + futureFiles)
>= storeConfigInfo.getBlockingFileCount(); >= storeConfigInfo.getBlockingFileCount();
candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting); candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting);
LOG.debug("Selecting compaction from " + candidateFiles.size() + " store files, " +
filesCompacting.size() + " compacting, " + candidateSelection.size() +
" eligible, " + storeConfigInfo.getBlockingFileCount() + " blocking");
long cfTtl = this.storeConfigInfo.getStoreFileTtl(); long cfTtl = this.storeConfigInfo.getStoreFileTtl();
if (!forceMajor) { if (!forceMajor) {
// If there are expired files, only select them so that compaction deletes them // If there are expired files, only select them so that compaction deletes them