HBASE-18949 Remove the CompactionRequest parameter in preCompactSelection

Signed-off-by: zhangduo <zhangduo@apache.org>
This commit is contained in:
Peter Somogyi 2017-10-05 09:59:28 -07:00 committed by zhangduo
parent ca62f769b6
commit 294f6b7860
5 changed files with 7 additions and 14 deletions

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
@ -188,11 +187,10 @@ public interface RegionObserver {
* @param store the store where compaction is being requested
* @param candidates the store files currently available for compaction
* @param tracker tracker used to track the life cycle of a compaction
* @param request the requested compaction
*/
default void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker,
CompactionRequest request) throws IOException {}
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker)
throws IOException {}
/**
* Called after the {@link StoreFile}s to compact have been selected from the available

View File

@ -1666,7 +1666,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
boolean override = false;
//TODO: is it correct way to get CompactionRequest?
override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc,
tracker, null, user);
tracker, user);
if (override) {
// Coprocessor is overriding normal file selection.
compaction.forceSelect(new CompactionRequestImpl(candidatesForCoproc));

View File

@ -569,18 +569,16 @@ public class RegionCoprocessorHost
* @param store The store where compaction is being requested
* @param candidates The currently available store files
* @param tracker used to track the life cycle of a compaction
* @param request the compaction request
* @param user the user
* @return If {@code true}, skip the normal selection process and use the current list
* @throws IOException
*/
public boolean preCompactSelection(final HStore store, final List<HStoreFile> candidates,
final CompactionLifeCycleTracker tracker, final CompactionRequest request,
final User user) throws IOException {
final CompactionLifeCycleTracker tracker, final User user) throws IOException {
return execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperation(user) {
@Override
public void call(RegionObserver observer) throws IOException {
observer.preCompactSelection(this, store, candidates, tracker, request);
observer.preCompactSelection(this, store, candidates, tracker);
}
});
}

View File

@ -203,8 +203,7 @@ public class SimpleRegionObserver implements RegionCoprocessor, RegionObserver {
@Override
public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker,
CompactionRequest request) throws IOException {
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
ctPreCompactSelect.incrementAndGet();
}

View File

@ -89,7 +89,6 @@ import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.security.EncryptionUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -731,8 +730,7 @@ public class TestMobCompactor {
@Override
public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker,
CompactionRequest request)
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker)
throws IOException {
int count = candidates.size();
if (count >= 2) {