You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2016/01/22 19:46:37 UTC

[2/2] hbase git commit: Revert "HBASE-14970 Backport HBASE-13082 and its sub-jira to branch-1 (Ram)" Causes TestHFileOutputFormat to fail every time. Messes up our 1.3 CI.

Revert "HBASE-14970 Backport HBASE-13082 and its sub-jira to branch-1 (Ram)"
Causes TestHFileOutputFormat to fail every time. Messes up our 1.3 CI.

This reverts commit 58521869b06a63894e422e9c9403e48b4b12f388.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a5228a0b
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a5228a0b
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a5228a0b

Branch: refs/heads/branch-1
Commit: a5228a0b4d8b4c6b9bab58e1eee015393edaaade
Parents: 37d15e0
Author: stack <st...@apache.org>
Authored: Fri Jan 22 10:45:51 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Jan 22 10:45:51 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/executor/EventType.java |   9 +-
 .../hadoop/hbase/executor/ExecutorType.java     |   3 +-
 .../hadoop/hbase/executor/ExecutorService.java  |   4 +-
 .../regionserver/ChangedReadersObserver.java    |   3 +-
 .../CompactedHFilesDischargeHandler.java        |  45 ---
 .../regionserver/CompactedHFilesDischarger.java | 110 -----
 .../regionserver/DefaultStoreFileManager.java   |  58 +--
 .../hadoop/hbase/regionserver/HRegion.java      |   5 -
 .../hbase/regionserver/HRegionServer.java       |  26 +-
 .../hadoop/hbase/regionserver/HStore.java       | 187 +++------
 .../hbase/regionserver/OnlineRegions.java       |   6 -
 .../regionserver/ReversedStoreScanner.java      |  21 +-
 .../apache/hadoop/hbase/regionserver/Store.java |  24 --
 .../hadoop/hbase/regionserver/StoreFile.java    |  67 +---
 .../hbase/regionserver/StoreFileManager.java    |  25 +-
 .../hbase/regionserver/StoreFileScanner.java    |  14 +-
 .../hadoop/hbase/regionserver/StoreScanner.java | 138 +++----
 .../regionserver/StripeStoreFileManager.java    |  82 +---
 .../compactions/CompactionConfiguration.java    |   7 -
 .../hadoop/hbase/MockRegionServerServices.java  |   5 -
 .../org/apache/hadoop/hbase/TestIOFencing.java  |  11 +
 .../TestZooKeeperTableArchiveClient.java        |  37 +-
 .../apache/hadoop/hbase/io/TestHeapSize.java    |   2 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java |  16 -
 .../hadoop/hbase/master/MockRegionServer.java   |   5 -
 .../master/cleaner/TestSnapshotFromMaster.java  |  15 -
 .../hbase/regionserver/MockStoreFile.java       |   5 -
 .../regionserver/TestEncryptionKeyRotation.java |  59 +--
 .../regionserver/TestHRegionReplayEvents.java   |  16 +-
 .../TestRegionMergeTransactionOnCluster.java    |  43 +-
 .../hbase/regionserver/TestRegionReplicas.java  |  22 +-
 .../hadoop/hbase/regionserver/TestStore.java    |  13 -
 .../hbase/regionserver/TestStoreScanner.java    |   4 +-
 .../TestStripeStoreFileManager.java             |  19 -
 .../hbase/regionserver/TestWideScanner.java     |   2 +-
 .../TestCompactedHFilesDischarger.java          | 398 -------------------
 36 files changed, 180 insertions(+), 1326 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
index a7759c5..ac76edb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
@@ -265,14 +265,7 @@ public enum EventType {
    *
    * RS_REGION_REPLICA_FLUSH
    */
-  RS_REGION_REPLICA_FLUSH   (82, ExecutorType.RS_REGION_REPLICA_FLUSH_OPS),
-
-  /**
-   * RS compacted files discharger <br>
-   *
-   * RS_COMPACTED_FILES_DISCHARGER
-   */
-  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER);
+  RS_REGION_REPLICA_FLUSH   (82, ExecutorType.RS_REGION_REPLICA_FLUSH_OPS);
 
   private final int code;
   private final ExecutorType executor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
index 5a16149..d0f6bee 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
@@ -46,8 +46,7 @@ public enum ExecutorType {
   RS_CLOSE_META              (25),
   RS_PARALLEL_SEEK           (26),
   RS_LOG_REPLAY_OPS          (27),
-  RS_REGION_REPLICA_FLUSH_OPS  (28),
-  RS_COMPACTED_FILES_DISCHARGER (29);
+  RS_REGION_REPLICA_FLUSH_OPS  (28);
 
   ExecutorType(int value) {}
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
index 018e173..42cca2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener;
 import org.apache.hadoop.hbase.monitoring.ThreadMonitoring;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -87,8 +86,7 @@ public class ExecutorService {
    * started with the same name, this throws a RuntimeException.
    * @param name Name of the service to start.
    */
-  @VisibleForTesting
-  public void startExecutorService(String name, int maxThreads) {
+  void startExecutorService(String name, int maxThreads) {
     if (this.executorMap.get(name) != null) {
       throw new RuntimeException("An executor service with the name " + name +
         " is already running!");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
index 0bc75e7..36b7559 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
@@ -20,7 +20,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
-import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
@@ -34,5 +33,5 @@ public interface ChangedReadersObserver {
    * Notify observers.
    * @throws IOException e
    */
-  void updateReaders(List<StoreFile> sfs) throws IOException;
+  void updateReaders() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java
deleted file mode 100644
index 02160d8..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.executor.EventHandler;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.regionserver.HStore;
-
-/**
- * Event handler that handles the removal and archival of the compacted hfiles
- */
-@InterfaceAudience.Private
-public class CompactedHFilesDischargeHandler extends EventHandler {
-
-  private HStore store;
-
-  public CompactedHFilesDischargeHandler(Server server, EventType eventType, HStore store) {
-    super(server, eventType);
-    this.store = store;
-  }
-
-  @Override
-  public void process() throws IOException {
-    this.store.closeAndArchiveCompactedFiles();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
deleted file mode 100644
index 9f6c65c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.executor.EventType;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * A chore service that periodically cleans up the compacted files when there are no active readers
- * using those compacted files and also helps in clearing the block cache with these compacted
- * file entries
- */
-@InterfaceAudience.Private
-public class CompactedHFilesDischarger extends ScheduledChore {
-  private static final Log LOG = LogFactory.getLog(CompactedHFilesDischarger.class);
-  private RegionServerServices regionServerServices;
-  // Default is to use executor
-  @VisibleForTesting
-  private boolean useExecutor = true;
-
-  /**
-   * @param period the period of time to sleep between each run
-   * @param stopper the stopper
-   * @param regionServerServices the region server that starts this chore
-   */
-  public CompactedHFilesDischarger(final int period, final Stoppable stopper,
-      final RegionServerServices regionServerServices) {
-    // Need to add the config classes
-    super("CompactedHFilesCleaner", stopper, period);
-    this.regionServerServices = regionServerServices;
-  }
-
-  /**
-   * @param period the period of time to sleep between each run
-   * @param stopper the stopper
-   * @param regionServerServices the region server that starts this chore
-   * @param useExecutor true if to use the region server's executor service, false otherwise
-   */
-  @VisibleForTesting
-  public CompactedHFilesDischarger(final int period, final Stoppable stopper,
-      final RegionServerServices regionServerServices, boolean useExecutor) {
-    // Need to add the config classes
-    this(period, stopper, regionServerServices);
-    this.useExecutor = useExecutor;
-  }
-
-  @Override
-  public void chore() {
-    if (regionServerServices == null) return;
-    List<Region> onlineRegions = regionServerServices.getOnlineRegions();
-    if (onlineRegions != null) {
-      for (Region region : onlineRegions) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(
-              "Started the compacted hfiles cleaner for the region " + region.getRegionInfo());
-        }
-        for (Store store : region.getStores()) {
-          try {
-            if (useExecutor && regionServerServices != null) {
-              CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler(
-                  (Server) regionServerServices, EventType.RS_COMPACTED_FILES_DISCHARGER,
-                  (HStore) store);
-              regionServerServices.getExecutorService().submit(handler);
-            } else {
-              // call synchronously if the RegionServerServices are not
-              // available
-              store.closeAndArchiveCompactedFiles();
-            }
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Completed archiving the compacted files for the region "
-                  + region.getRegionInfo() + " under the store " + store.getColumnFamilyName());
-            }
-          } catch (Exception e) {
-            LOG.error("Exception while trying to close and archive the comapcted store "
-                + "files of the store  " + store.getColumnFamilyName() + " in the" + " region "
-                + region.getRegionInfo(), e);
-          }
-        }
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(
-              "Completed the compacted hfiles cleaner for the region " + region.getRegionInfo());
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
index 6611202..6000f01 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
@@ -54,13 +54,6 @@ class DefaultStoreFileManager implements StoreFileManager {
    * is atomically replaced when its contents change.
    */
   private volatile ImmutableList<StoreFile> storefiles = null;
-  /**
-   * List of compacted files inside this store that needs to be excluded in reads
-   * because further new reads will be using only the newly created files out of compaction.
-   * These compacted files will be deleted/cleared once all the existing readers on these
-   * compacted files are done.
-   */
-  private volatile List<StoreFile> compactedfiles = null;
 
   public DefaultStoreFileManager(KVComparator kvComparator, Configuration conf,
       CompactionConfiguration comConf) {
@@ -82,11 +75,6 @@ class DefaultStoreFileManager implements StoreFileManager {
   }
 
   @Override
-  public Collection<StoreFile> getCompactedfiles() {
-    return compactedfiles;
-  }
-
-  @Override
   public void insertNewFiles(Collection<StoreFile> sfs) throws IOException {
     ArrayList<StoreFile> newFiles = new ArrayList<StoreFile>(storefiles);
     newFiles.addAll(sfs);
@@ -101,55 +89,19 @@ class DefaultStoreFileManager implements StoreFileManager {
   }
 
   @Override
-  public Collection<StoreFile> clearCompactedFiles() {
-    List<StoreFile> result = compactedfiles;
-    compactedfiles = new ArrayList<StoreFile>();
-    return result;
-  }
-
-  @Override
   public final int getStorefileCount() {
     return storefiles.size();
   }
 
   @Override
   public void addCompactionResults(
-    Collection<StoreFile> newCompactedfiles, Collection<StoreFile> results) {
+    Collection<StoreFile> compactedFiles, Collection<StoreFile> results) {
     ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles);
-    newStoreFiles.removeAll(newCompactedfiles);
+    newStoreFiles.removeAll(compactedFiles);
     if (!results.isEmpty()) {
       newStoreFiles.addAll(results);
     }
     sortAndSetStoreFiles(newStoreFiles);
-    ArrayList<StoreFile> updatedCompactedfiles = null;
-    if (this.compactedfiles != null) {
-      updatedCompactedfiles = new ArrayList<StoreFile>(this.compactedfiles);
-      updatedCompactedfiles.addAll(newCompactedfiles);
-    } else {
-      updatedCompactedfiles = new ArrayList<StoreFile>(newCompactedfiles);
-    }
-    markCompactedAway(newCompactedfiles);
-    this.compactedfiles = sortCompactedfiles(updatedCompactedfiles);
-  }
-
-  // Mark the files as compactedAway once the storefiles and compactedfiles list is finalised
-  // Let a background thread close the actual reader on these compacted files and also
-  // ensure to evict the blocks from block cache so that they are no longer in
-  // cache
-  private void markCompactedAway(Collection<StoreFile> compactedFiles) {
-    for (StoreFile file : compactedFiles) {
-      file.markCompactedAway();
-    }
-  }
-
-  @Override
-  public void removeCompactedFiles(Collection<StoreFile> removedCompactedfiles) throws IOException {
-    ArrayList<StoreFile> updatedCompactedfiles = null;
-    if (this.compactedfiles != null) {
-      updatedCompactedfiles = new ArrayList<StoreFile>(this.compactedfiles);
-      updatedCompactedfiles.removeAll(removedCompactedfiles);
-      this.compactedfiles = sortCompactedfiles(updatedCompactedfiles);
-    }
   }
 
   @Override
@@ -214,12 +166,6 @@ class DefaultStoreFileManager implements StoreFileManager {
     storefiles = ImmutableList.copyOf(storeFiles);
   }
 
-  private List<StoreFile> sortCompactedfiles(List<StoreFile> storefiles) {
-    // Sorting may not be really needed here for the compacted files?
-    Collections.sort(storefiles, StoreFile.Comparators.SEQ_ID);
-    return new ArrayList<StoreFile>(storefiles);
-  }
-
   @Override
   public double getCompactionPressure() {
     int storefileCount = getStorefileCount();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 1c35832..899f2fc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1533,7 +1533,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       if (this.metricsRegionWrapper != null) {
         Closeables.closeQuietly(this.metricsRegionWrapper);
       }
-
       status.markComplete("Closed");
       LOG.info("Closed " + this);
       return result;
@@ -6787,10 +6786,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       dstRegion.getRegionFileSystem().logFileSystemState(LOG);
     }
 
-    // clear the compacted files if any
-    for (Store s : dstRegion.getStores()) {
-      s.closeAndArchiveCompactedFiles();
-    }
     if (dstRegion.getRegionFileSystem().hasReferences(dstRegion.getTableDesc())) {
       throw new IOException("Merged region " + dstRegion
           + " still has references after the compaction, is compaction canceled?");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 7287f78..7351fa8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -133,7 +133,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Repor
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
@@ -480,8 +479,6 @@ public class HRegionServer extends HasThread implements
    */
   protected final ConfigurationManager configurationManager;
 
-  private CompactedHFilesDischarger compactedFileDischarger;
-
   /**
    * Starts a HRegionServer at the default location.
    * @param conf
@@ -614,16 +611,6 @@ public class HRegionServer extends HasThread implements
         }
       });
     }
-    // Create the CompactedFileDischarger chore service. This chore helps to
-    // remove the compacted files
-    // that will no longer be used in reads.
-    // Default is 2 mins. The default value for TTLCleaner is 5 mins so we set this to
-    // 2 mins so that compacted files can be archived before the TTLCleaner runs
-    int cleanerInterval = conf
-        .getInt(CompactionConfiguration.HBASE_HFILE_COMPACTION_DISCHARGER_INTERVAL, 2 * 60 * 1000);
-    this.compactedFileDischarger =
-        new CompactedHFilesDischarger(cleanerInterval, (Stoppable)this, (RegionServerServices)this);
-    choreService.scheduleChore(compactedFileDischarger);
   }
 
   /*
@@ -1721,9 +1708,7 @@ public class HRegionServer extends HasThread implements
     }
     this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, conf.getInt(
        "hbase.regionserver.wal.max.splitters", SplitLogWorkerCoordination.DEFAULT_MAX_SPLITTERS));
-    // Start the threads for compacted files discharger
-    this.service.startExecutorService(ExecutorType.RS_COMPACTED_FILES_DISCHARGER,
-      conf.getInt(CompactionConfiguration.HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT, 10));
+
     if (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) {
       this.service.startExecutorService(ExecutorType.RS_REGION_REPLICA_FLUSH_OPS,
         conf.getInt("hbase.regionserver.region.replica.flusher.threads",
@@ -2747,15 +2732,6 @@ public class HRegionServer extends HasThread implements
     return coprocessors.toArray(new String[coprocessors.size()]);
   }
 
-  @Override
-  public List<Region> getOnlineRegions() {
-    List<Region> allRegions = new ArrayList<Region>();
-    synchronized (this.onlineRegions) {
-      // Return a clone copy of the onlineRegions
-      allRegions.addAll(onlineRegions.values());
-    }
-    return allRegions;
-  }
   /**
    * Try to close the region, logs a warning on failure but continues.
    * @param region Region to close

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index cb62c95..a19407a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
@@ -272,6 +273,7 @@ public class HStore implements Store {
           "hbase.hstore.flush.retries.number must be > 0, not "
               + flushRetriesNumber);
     }
+
     // Crypto context for new store files
     String cipherName = family.getEncryptionType();
     if (cipherName != null) {
@@ -529,15 +531,14 @@ public class HStore implements Store {
         try {
           Future<StoreFile> future = completionService.take();
           StoreFile storeFile = future.get();
-          if (storeFile != null) {
-            long length = storeFile.getReader().length();
-            this.storeSize += length;
-            this.totalUncompressedBytes += storeFile.getReader().getTotalUncompressedBytes();
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("loaded " + storeFile.toStringDetailed());
-            }
-            results.add(storeFile);
+          long length = storeFile.getReader().length();
+          this.storeSize += length;
+          this.totalUncompressedBytes +=
+              storeFile.getReader().getTotalUncompressedBytes();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("loaded " + storeFile.toStringDetailed());
           }
+          results.add(storeFile);
         } catch (InterruptedException e) {
           if (ioe == null) ioe = new InterruptedIOException(e.getMessage());
         } catch (ExecutionException e) {
@@ -635,7 +636,8 @@ public class HStore implements Store {
       region.getMVCC().advanceTo(this.getMaxSequenceId());
     }
 
-    completeCompaction(toBeRemovedStoreFiles);
+    // notify scanners, close file readers, and recompute store size
+    completeCompaction(toBeRemovedStoreFiles, false);
   }
 
   private StoreFile createStoreFileAndReader(final Path p) throws IOException {
@@ -812,6 +814,7 @@ public class HStore implements Store {
       // the lock.
       this.lock.writeLock().unlock();
     }
+    notifyChangedReadersObservers();
     LOG.info("Loaded HFile " + sf.getFileInfo() + " into store '" + getColumnFamilyName());
     if (LOG.isTraceEnabled()) {
       String traceMessage = "BULK LOAD time,size,store size,store files ["
@@ -827,12 +830,7 @@ public class HStore implements Store {
     try {
       // Clear so metrics doesn't find them.
       ImmutableCollection<StoreFile> result = storeEngine.getStoreFileManager().clearFiles();
-      Collection<StoreFile> compactedfiles =
-          storeEngine.getStoreFileManager().clearCompactedFiles();
-      // clear the compacted files
-      if (compactedfiles != null && !compactedfiles.isEmpty()) {
-        removeCompactedfiles(compactedfiles);
-      }
+
       if (!result.isEmpty()) {
         // initialize the thread pool for closing store files in parallel.
         ThreadPoolExecutor storeFileCloserThreadPool = this.region
@@ -847,7 +845,7 @@ public class HStore implements Store {
             @Override
             public Void call() throws IOException {
               boolean evictOnClose = 
-                  cacheConf != null? cacheConf.shouldEvictOnClose(): true;
+                  cacheConf != null? cacheConf.shouldEvictOnClose(): true; 
               f.closeReader(evictOnClose);
               return null;
             }
@@ -1069,8 +1067,10 @@ public class HStore implements Store {
       // the lock.
       this.lock.writeLock().unlock();
     }
-    // notify to be called here - only in case of flushes
-    notifyChangedReadersObservers(sfs);
+
+    // Tell listeners of the change in readers.
+    notifyChangedReadersObservers();
+
     if (LOG.isTraceEnabled()) {
       long totalSize = 0;
       for (StoreFile sf : sfs) {
@@ -1088,9 +1088,9 @@ public class HStore implements Store {
    * Notify all observers that set of Readers has changed.
    * @throws IOException
    */
-  private void notifyChangedReadersObservers(List<StoreFile> sfs) throws IOException {
-    for (ChangedReadersObserver o : this.changedReaderObservers) {
-      o.updateReaders(sfs);
+  private void notifyChangedReadersObservers() throws IOException {
+    for (ChangedReadersObserver o: this.changedReaderObservers) {
+      o.updateReaders();
     }
   }
 
@@ -1130,30 +1130,6 @@ public class HStore implements Store {
   }
 
   @Override
-  public List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
-      boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
-      byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException {
-    List<KeyValueScanner> memStoreScanners = null;
-    if (includeMemstoreScanner) {
-      this.lock.readLock().lock();
-      try {
-        memStoreScanners = this.memstore.getScanners(readPt);
-      } finally {
-        this.lock.readLock().unlock();
-      }
-    }
-    List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(files,
-      cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore());
-    List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>(sfScanners.size() + 1);
-    scanners.addAll(sfScanners);
-    // Then the memstore scanners
-    if (memStoreScanners != null) {
-      scanners.addAll(memStoreScanners);
-    }
-    return scanners;
-  }
-
-  @Override
   public void addChangedReaderObserver(ChangedReadersObserver o) {
     this.changedReaderObservers.add(o);
   }
@@ -1272,7 +1248,7 @@ public class HStore implements Store {
         compactedCellsSize += getCompactionProgress().totalCompactedSize;
       }
       // At this point the store will use new files for all new scanners.
-      completeCompaction(filesToCompact); // update store size.
+      completeCompaction(filesToCompact, true); // Archive old files & update store size.
 
       logCompactionEndMessage(cr, sfs, compactionStartTime);
       return sfs;
@@ -1460,7 +1436,7 @@ public class HStore implements Store {
       LOG.info("Replaying compaction marker, replacing input files: " +
           inputStoreFiles + " with output files : " + outputStoreFiles);
       this.replaceStoreFiles(inputStoreFiles, outputStoreFiles);
-      this.completeCompaction(inputStoreFiles);
+      this.completeCompaction(inputStoreFiles, removeFiles);
     }
   }
 
@@ -1512,7 +1488,7 @@ public class HStore implements Store {
           this.getCoprocessorHost().postCompact(this, sf, null);
         }
         replaceStoreFiles(filesToCompact, Lists.newArrayList(sf));
-        completeCompaction(filesToCompact);
+        completeCompaction(filesToCompact, true);
       }
     } finally {
       synchronized (filesCompacting) {
@@ -1795,7 +1771,32 @@ public class HStore implements Store {
   @VisibleForTesting
   protected void completeCompaction(final Collection<StoreFile> compactedFiles, boolean removeFiles)
       throws IOException {
-    LOG.debug("Completing compaction...");
+    try {
+      // Do not delete old store files until we have sent out notification of
+      // change in case old files are still being accessed by outstanding scanners.
+      // Don't do this under writeLock; see HBASE-4485 for a possible deadlock
+      // scenario that could have happened if continue to hold the lock.
+      notifyChangedReadersObservers();
+      // At this point the store will use new files for all scanners.
+
+      // let the archive util decide if we should archive or delete the files
+      LOG.debug("Removing store files after compaction...");
+      boolean evictOnClose = 
+          cacheConf != null? cacheConf.shouldEvictOnClose(): true; 
+      for (StoreFile compactedFile : compactedFiles) {
+        compactedFile.closeReader(evictOnClose);
+      }
+      if (removeFiles) {
+        this.fs.removeStoreFiles(this.getColumnFamilyName(), compactedFiles);
+      }
+    } catch (IOException e) {
+      e = RemoteExceptionHandler.checkIOException(e);
+      LOG.error("Failed removing compacted files in " + this +
+        ". Files we were trying to remove are " + compactedFiles.toString() +
+        "; some of them may have been already removed", e);
+    }
+
+    // 4. Compute new store size
     this.storeSize = 0L;
     this.totalUncompressedBytes = 0L;
     for (StoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
@@ -2489,92 +2490,4 @@ public class HStore implements Store {
   public boolean isPrimaryReplicaStore() {
 	   return getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID;
   }
-
-  @Override
-  public void closeAndArchiveCompactedFiles() throws IOException {
-    lock.readLock().lock();
-    Collection<StoreFile> copyCompactedfiles = null;
-    try {
-      Collection<StoreFile> compactedfiles =
-          this.getStoreEngine().getStoreFileManager().getCompactedfiles();
-      if (compactedfiles != null && compactedfiles.size() != 0) {
-        // Do a copy under read lock
-        copyCompactedfiles = new ArrayList<StoreFile>(compactedfiles);
-      } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("No compacted files to archive");
-          return;
-        }
-      }
-    } finally {
-      lock.readLock().unlock();
-    }
-    if (copyCompactedfiles != null && !copyCompactedfiles.isEmpty()) {
-      removeCompactedfiles(copyCompactedfiles);
-    }
-  }
-
-  /**
-   * Archives and removes the compacted files
-   * @param compactedfiles The compacted files in this store that are not active in reads
-   * @throws IOException
-   */
-  private void removeCompactedfiles(Collection<StoreFile> compactedfiles)
-      throws IOException {
-    final List<StoreFile> filesToRemove = new ArrayList<StoreFile>(compactedfiles.size());
-    for (final StoreFile file : compactedfiles) {
-      synchronized (file) {
-        try {
-          StoreFile.Reader r = file.getReader();
-          if (r == null) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("The file " + file + " was closed but still not archived.");
-            }
-            filesToRemove.add(file);
-          }
-          if (r != null && r.isCompactedAway() && !r.isReferencedInReads()) {
-            // Even if deleting fails we need not bother as any new scanners won't be
-            // able to use the compacted file as the status is already compactedAway
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Closing and archiving the file " + file.getPath());
-            }
-            r.close(true);
-            // Just close and return
-            filesToRemove.add(file);
-          }
-        } catch (Exception e) {
-          LOG.error(
-            "Exception while trying to close the compacted store file " + file.getPath().getName());
-        }
-      }
-    }
-    if (this.isPrimaryReplicaStore()) {
-      // Only the primary region is allowed to move the file to archive.
-      // The secondary region does not move the files to archive. Any active reads from
-      // the secondary region will still work because the file as such has active readers on it.
-      if (!filesToRemove.isEmpty()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Moving the files " + filesToRemove + " to archive");
-        }
-        // Only if this is successful it has to be removed
-        this.fs.removeStoreFiles(this.getFamily().getNameAsString(), filesToRemove);
-      }
-    }
-    if (!filesToRemove.isEmpty()) {
-      // Clear the compactedfiles from the store file manager
-      clearCompactedfiles(filesToRemove);
-    }
-  }
-
-  private void clearCompactedfiles(final List<StoreFile> filesToRemove) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Clearing the compacted file " + filesToRemove + " from this store");
-    }
-    try {
-      lock.writeLock().lock();
-      this.getStoreEngine().getStoreFileManager().removeCompactedFiles(filesToRemove);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
index 310108c..60fc9fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
@@ -67,10 +67,4 @@ public interface OnlineRegions extends Server {
     * @throws java.io.IOException
     */
    List<Region> getOnlineRegions(TableName tableName) throws IOException;
-
-   /**
-    * Get all online regions in this RS.
-    * @return List of online Region
-    */
-   List<Region> getOnlineRegions();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
index 0f12b0a..e319f90 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
@@ -124,15 +124,24 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
 
   @Override
   public boolean seekToPreviousRow(Cell key) throws IOException {
-    boolean flushed = checkFlushed();
-    checkReseek(flushed);
-    return this.heap.seekToPreviousRow(key);
+    lock.lock();
+    try {
+      checkReseek();
+      return this.heap.seekToPreviousRow(key);
+    } finally {
+      lock.unlock();
+    }
+
   }
   
   @Override
   public boolean backwardSeek(Cell key) throws IOException {
-    boolean flushed = checkFlushed();
-    checkReseek(flushed);
-    return this.heap.backwardSeek(key);
+    lock.lock();
+    try {
+      checkReseek();
+      return this.heap.backwardSeek(key);
+    } finally {
+      lock.unlock();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index ddcd4e9..9f17526 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -105,25 +105,6 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
     byte[] stopRow,
     long readPt
   ) throws IOException;
-
-  /**
-   * Create scanners on the given files and if needed on the memstore with no filtering based on TTL
-   * (that happens further down the line).
-   * @param files the list of files on which the scanners has to be created
-   * @param cacheBlocks cache the blocks or not
-   * @param isGet true if it is get, false if not
-   * @param usePread true to use pread, false if not
-   * @param isCompaction true if the scanner is created for compaction
-   * @param matcher the scan query matcher
-   * @param startRow the start row
-   * @param stopRow the stop row
-   * @param readPt the read point of the current scan
-   * @param includeMemstoreScanner true if memstore has to be included
-   * @return scanners on the given files and on the memstore if specified
-   */
-   List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks, boolean isGet,
-          boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
-          byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException;
   
   ScanInfo getScanInfo();
 
@@ -499,9 +480,4 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
   void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException;
 
   boolean isPrimaryReplicaStore();
-
-  /**
-   * Closes and archives the compacted files under this store
-   */
-  void closeAndArchiveCompactedFiles() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 57a272e..46a64f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -30,8 +30,6 @@ import java.util.Map;
 import java.util.SortedSet;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -63,7 +61,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.WritableUtils;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -371,19 +368,6 @@ public class StoreFile {
     return bulkLoadedHFile || metadataMap.containsKey(BULKLOAD_TIME_KEY);
   }
 
-  @VisibleForTesting
-  public boolean isCompactedAway() {
-    if (this.reader != null) {
-      return this.reader.isCompactedAway();
-    }
-    return true;
-  }
-
-  @VisibleForTesting
-  public int getRefCount() {
-    return this.reader.refCount.get();
-  }
-
   /**
    * Return the timestamp at which this bulk load file was generated.
    */
@@ -553,15 +537,6 @@ public class StoreFile {
   }
 
   /**
-   * Marks the status of the file as compactedAway.
-   */
-  public void markCompactedAway() {
-    if (this.reader != null) {
-      this.reader.markCompactedAway();
-    }
-  }
-
-  /**
    * Delete this file
    * @throws IOException
    */
@@ -1097,12 +1072,6 @@ public class StoreFile {
     private byte[] lastBloomKey;
     private long deleteFamilyCnt = -1;
     private boolean bulkLoadResult = false;
-    // Counter that is incremented every time a scanner is created on the
-    // store file. It is decremented when the scan on the store file is
-    // done.
-    private AtomicInteger refCount = new AtomicInteger(0);
-    // Indicates if the file got compacted
-    private volatile boolean compactedAway = false;
 
     public Reader(FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf)
         throws IOException {
@@ -1110,10 +1079,6 @@ public class StoreFile {
       bloomFilterType = BloomType.NONE;
     }
 
-    void markCompactedAway() {
-      this.compactedAway = true;
-    }
-
     public Reader(FileSystem fs, Path path, FSDataInputStreamWrapper in, long size,
         CacheConfig cacheConf, Configuration conf) throws IOException {
       reader = HFile.createReader(fs, path, in, size, cacheConf, conf);
@@ -1165,36 +1130,12 @@ public class StoreFile {
     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
                                                boolean pread,
                                                boolean isCompaction, long readPt) {
-      // Increment the ref count
-      refCount.incrementAndGet();
       return new StoreFileScanner(this,
                                  getScanner(cacheBlocks, pread, isCompaction),
                                  !isCompaction, reader.hasMVCCInfo(), readPt);
     }
 
     /**
-     * Decrement the ref count associated with the reader when ever a scanner associated
-     * with the reader is closed
-     */
-    void decrementRefCount() {
-      refCount.decrementAndGet();
-    }
-
-    /**
-     * @return true if the file is still used in reads
-     */
-    public boolean isReferencedInReads() {
-      return refCount.get() != 0;
-    }
- 
-    /**
-     * @return true if the file is compacted
-     */
-    public boolean isCompactedAway() {
-      return this.compactedAway;
-    }
-
-    /**
      * Warning: Do not write further code which depends on this call. Instead
      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
      * which is the preferred way to scan a store with higher level concepts.
@@ -1679,13 +1620,7 @@ public class StoreFile {
     private static class GetFileSize implements Function<StoreFile, Long> {
       @Override
       public Long apply(StoreFile sf) {
-        if (sf.getReader() != null) {
-          return sf.getReader().length();
-        } else {
-          // the reader may be null for the compacted files and if the archiving
-          // had failed.
-          return -1L;
-        }
+        return sf.getReader().length();
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
index 7e70547..11993db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
@@ -53,7 +53,7 @@ public interface StoreFileManager {
   void insertNewFiles(Collection<StoreFile> sfs) throws IOException;
 
   /**
-   * Adds only the new compaction results into the structure.
+   * Adds compaction results into the structure.
    * @param compactedFiles The input files for the compaction.
    * @param results The resulting files for the compaction.
    */
@@ -61,26 +61,12 @@ public interface StoreFileManager {
       Collection<StoreFile> compactedFiles, Collection<StoreFile> results) throws IOException;
 
   /**
-   * Remove the compacted files
-   * @param compactedFiles the list of compacted files
-   * @throws IOException
-   */
-  void removeCompactedFiles(Collection<StoreFile> compactedFiles) throws IOException;
-
-  /**
    * Clears all the files currently in use and returns them.
    * @return The files previously in use.
    */
   ImmutableCollection<StoreFile> clearFiles();
 
   /**
-   * Clears all the compacted files and returns them. This method is expected to be
-   * accessed single threaded.
-   * @return The files compacted previously.
-   */
-  Collection<StoreFile> clearCompactedFiles();
-
-  /**
    * Gets the snapshot of the store files currently in use. Can be used for things like metrics
    * and checks; should not assume anything about relations between store files in the list.
    * @return The list of StoreFiles.
@@ -88,15 +74,6 @@ public interface StoreFileManager {
   Collection<StoreFile> getStorefiles();
 
   /**
-   * List of compacted files inside this store that needs to be excluded in reads
-   * because further new reads will be using only the newly created files out of compaction.
-   * These compacted files will be deleted/cleared once all the existing readers on these
-   * compacted files are done.
-   * @return the list of compacted files
-   */
-  Collection<StoreFile> getCompactedfiles();
-
-  /**
    * Returns the number of files currently in use.
    * @return The number of files.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
index 4055188..1d2f7e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
@@ -51,7 +51,6 @@ public class StoreFileScanner implements KeyValueScanner {
   private final StoreFile.Reader reader;
   private final HFileScanner hfs;
   private Cell cur = null;
-  private boolean closed = false;
 
   private boolean realSeekDone;
   private boolean delayedReseek;
@@ -172,7 +171,7 @@ public class StoreFileScanner implements KeyValueScanner {
     try {
       try {
         if(!seekAtOrAfter(hfs, key)) {
-          this.cur = null;
+          close();
           return false;
         }
 
@@ -199,7 +198,7 @@ public class StoreFileScanner implements KeyValueScanner {
     try {
       try {
         if (!reseekAtOrAfter(hfs, key)) {
-          this.cur = null;
+          close();
           return false;
         }
         setCurrentCell(hfs.getKeyValue());
@@ -245,6 +244,7 @@ public class StoreFileScanner implements KeyValueScanner {
     }
 
     if (cur == null) {
+      close();
       return false;
     }
 
@@ -252,12 +252,8 @@ public class StoreFileScanner implements KeyValueScanner {
   }
 
   public void close() {
+    // Nothing to close on HFileScanner?
     cur = null;
-    if (closed) return;    
-    if (this.reader != null) {
-      this.reader.decrementRefCount();
-    }
-    closed = true;
   }
 
   /**
@@ -458,7 +454,7 @@ public class StoreFileScanner implements KeyValueScanner {
           if (seekCount != null) seekCount.incrementAndGet();
           if (!hfs.seekBefore(seekKey.getBuffer(), seekKey.getKeyOffset(),
               seekKey.getKeyLength())) {
-            this.cur = null;
+            close();
             return false;
           }
           KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getKeyValue()

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index ddf0721..a026a40 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -121,14 +121,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   // A flag whether use pread for scan
   private boolean scanUsePread = false;
-  // Indicates whether there was flush during the course of the scan
-  private volatile boolean flushed = false;
-  // generally we get one file from a flush
-  private List<StoreFile> flushedStoreFiles = new ArrayList<StoreFile>(1);
-  // The current list of scanners
-  private List<KeyValueScanner> currentScanners = new ArrayList<KeyValueScanner>();
-  // flush update lock
-  private ReentrantLock flushLock = new ReentrantLock();
+  protected ReentrantLock lock = new ReentrantLock();
   
   private final long readPt;
 
@@ -173,9 +166,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
      }
   }
 
-  protected void addCurrentScanners(List<? extends KeyValueScanner> scanners) {
-    this.currentScanners.addAll(scanners);
-  }
   /**
    * Opens a scanner across memstore, snapshot, and all StoreFiles. Assumes we
    * are not in a compaction.
@@ -213,7 +203,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
     // set rowOffset
     this.storeOffset = scan.getRowOffsetPerColumnFamily();
-    addCurrentScanners(scanners);
+
     // Combine all seeked scanners with a heap
     resetKVHeap(scanners, store.getComparator());
   }
@@ -270,7 +260,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
     // Seek all scanners to the initial key
     seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
-    addCurrentScanners(scanners);
+
     // Combine all seeked scanners with a heap
     resetKVHeap(scanners, store.getComparator());
   }
@@ -309,7 +299,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     }
     // Seek all scanners to the initial key
     seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
-    addCurrentScanners(scanners);
     resetKVHeap(scanners, scanInfo.getComparator());
   }
 
@@ -403,8 +392,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
       if (kvs.shouldUseScanner(scan, store, expiredTimestampCutoff)) {
         scanners.add(kvs);
-      } else {
-        kvs.close();
       }
     }
     return scanners;
@@ -412,10 +399,15 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   @Override
   public Cell peek() {
+    lock.lock();
+    try {
     if (this.heap == null) {
       return this.lastTop;
     }
     return this.heap.peek();
+    } finally {
+      lock.unlock();
+    }
   }
 
   @Override
@@ -426,6 +418,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   @Override
   public void close() {
+    lock.lock();
+    try {
     if (this.closing) return;
     this.closing = true;
     // Under test, we dont have a this.store
@@ -435,14 +429,21 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       this.heap.close();
     this.heap = null; // CLOSED!
     this.lastTop = null; // If both are null, we are closed.
+    } finally {
+      lock.unlock();
+    }
   }
 
   @Override
   public boolean seek(Cell key) throws IOException {
-    boolean flushed = checkFlushed();
+    lock.lock();
+    try {
     // reset matcher state, in case that underlying store changed
-    checkReseek(flushed);
+    checkReseek();
     return this.heap.seek(key);
+    } finally {
+      lock.unlock();
+    }
   }
 
   @Override
@@ -458,11 +459,13 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
    */
   @Override
   public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws IOException {
+    lock.lock();
+
+    try {
     if (scannerContext == null) {
       throw new IllegalArgumentException("Scanner context cannot be null");
     }
-    boolean flushed = checkFlushed();
-    if (checkReseek(flushed)) {
+    if (checkReseek()) {
       return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
     }
 
@@ -640,6 +643,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     // No more keys
     close();
     return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
+    } finally {
+      lock.unlock();
+    }
   }
 
   /*
@@ -676,18 +682,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   // Implementation of ChangedReadersObserver
   @Override
-  public void updateReaders(List<StoreFile> sfs) throws IOException {
-    flushed = true;
-    flushLock.lock();
+  public void updateReaders() throws IOException {
+    lock.lock();
     try {
-      flushedStoreFiles.addAll(sfs);
-    } finally {
-      flushLock.unlock();
-    }
-  }
-
-  // Implementation of ChangedReadersObserver
-  protected void nullifyCurrentHeap() throws IOException {
     if (this.closing) return;
 
     // All public synchronized API calls will call 'checkReseek' which will cause
@@ -698,7 +695,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     if (this.heap == null) return;
 
     // this could be null.
-    this.lastTop = this.heap.peek();
+    this.lastTop = this.peek();
 
     //DebugPrint.println("SS updateReaders, topKey = " + lastTop);
 
@@ -707,16 +704,18 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     this.heap = null; // the re-seeks could be slow (access HDFS) free up memory ASAP
 
     // Let the next() call handle re-creating and seeking
+    } finally {
+      lock.unlock();
+    }
   }
 
   /**
-   * @param flushed indicates if there was a flush
    * @return true if top of heap has changed (and KeyValueHeap has to try the
    *         next KV)
    * @throws IOException
    */
-  protected boolean checkReseek(boolean flushed) throws IOException {
-    if (flushed && this.lastTop != null) {
+  protected boolean checkReseek() throws IOException {
+    if (this.heap == null && this.lastTop != null) {
       resetScannerStack(this.lastTop);
       if (this.heap.peek() == null
           || store.getComparator().compareRows(this.lastTop, this.heap.peek()) != 0) {
@@ -732,37 +731,21 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   }
 
   protected void resetScannerStack(Cell lastTopKey) throws IOException {
+    if (heap != null) {
+      throw new RuntimeException("StoreScanner.reseek run on an existing heap!");
+    }
+
     /* When we have the scan object, should we not pass it to getScanners()
      * to get a limited set of scanners? We did so in the constructor and we
-     * could have done it now by storing the scan object from the constructor
-     */
-
-    final boolean isCompaction = false;
-    boolean usePread = get || scanUsePread;
-    List<KeyValueScanner> scanners = null;
-    try {
-      flushLock.lock();
-      scanners = selectScannersFrom(store.getScanners(flushedStoreFiles, cacheBlocks, get, usePread,
-        isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt, true));
-      // Clear the current set of flushed store files so that they don't get added again
-      flushedStoreFiles.clear();
-    } finally {
-      flushLock.unlock();
-    }
+     * could have done it now by storing the scan object from the constructor */
+    List<KeyValueScanner> scanners = getScannersNoCompaction();
 
-    // Seek the new scanners to the last key
+    // Seek all scanners to the initial key
     seekScanners(scanners, lastTopKey, false, parallelSeekEnabled);
-    // remove the older memstore scanner
-    for (int i = 0; i < currentScanners.size(); i++) {
-      if (!currentScanners.get(i).isFileScanner()) {
-        currentScanners.remove(i);
-        break;
-      }
-    }
-    // add the newly created scanners on the flushed files and the current active memstore scanner
-    addCurrentScanners(scanners);
+
     // Combine all seeked scanners with a heap
-    resetKVHeap(this.currentScanners, store.getComparator());
+    resetKVHeap(scanners, store.getComparator());
+
     // Reset the state of the Query Matcher and set to top row.
     // Only reset and call setRow if the row changes; avoids confusing the
     // query matcher if scanning intra-row.
@@ -813,36 +796,19 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   @Override
   public boolean reseek(Cell kv) throws IOException {
-    boolean flushed = checkFlushed();
-    // Heap will not be null, if this is called from next() which.
-    // If called from RegionScanner.reseek(...) make sure the scanner
-    // stack is reset if needed.
-    checkReseek(flushed);
+    lock.lock();
+    try {
+    //Heap will not be null, if this is called from next() which.
+    //If called from RegionScanner.reseek(...) make sure the scanner
+    //stack is reset if needed.
+    checkReseek();
     if (explicitColumnQuery && lazySeekEnabledGlobally) {
       return heap.requestSeek(kv, true, useRowColBloom);
     }
     return heap.reseek(kv);
-  }
-
-  protected boolean checkFlushed() {
-    // check the var without any lock. Suppose even if we see the old
-    // value here still it is ok to continue because we will not be resetting
-    // the heap but will continue with the referenced memstore's snapshot. For compactions
-    // any way we don't need the updateReaders at all to happen as we still continue with 
-    // the older files
-    if (flushed) {
-      // If there is a flush and the current scan is notified on the flush ensure that the 
-      // scan's heap gets reset and we do a seek on the newly flushed file.
-      if(!this.closing) {
-        this.lastTop = this.peek();
-      } else {
-        return false;
-      }
-      // reset the flag
-      flushed = false;
-      return true;
+    } finally {
+      lock.unlock();
     }
-    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
index 5b79790..1bbaefe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
@@ -104,7 +104,6 @@ public class StripeStoreFileManager
 
     /** Cached list of all files in the structure, to return from some calls */
     public ImmutableList<StoreFile> allFilesCached = ImmutableList.<StoreFile>of();
-    private ImmutableList<StoreFile> allCompactedFilesCached = ImmutableList.<StoreFile>of();
   }
   private State state = null;
 
@@ -140,14 +139,8 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public Collection<StoreFile> getCompactedfiles() {
-    return state.allCompactedFilesCached;
-  }
-
-  @Override
   public void insertNewFiles(Collection<StoreFile> sfs) throws IOException {
     CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(true);
-    // Passing null does not cause NPE??
     cmc.mergeResults(null, sfs);
     debugDumpState("Added new files");
   }
@@ -162,13 +155,6 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public ImmutableCollection<StoreFile> clearCompactedFiles() {
-    ImmutableCollection<StoreFile> result = state.allCompactedFilesCached;
-    this.state = new State();
-    return result;
-  }
-
-  @Override
   public int getStorefileCount() {
     return state.allFilesCached.size();
   }
@@ -318,31 +304,9 @@ public class StripeStoreFileManager
     // copies and apply the result at the end.
     CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(false);
     cmc.mergeResults(compactedFiles, results);
-    markCompactedAway(compactedFiles);
     debugDumpState("Merged compaction results");
   }
 
-  // Mark the files as compactedAway once the storefiles and compactedfiles list is finalised
-  // Let a background thread close the actual reader on these compacted files and also
-  // ensure to evict the blocks from block cache so that they are no longer in
-  // cache
-  private void markCompactedAway(Collection<StoreFile> compactedFiles) {
-    for (StoreFile file : compactedFiles) {
-      file.markCompactedAway();
-    }
-  }
-
-  @Override
-  public void removeCompactedFiles(Collection<StoreFile> compactedFiles) throws IOException {
-    // See class comment for the assumptions we make here.
-    LOG.debug("Attempting to delete compaction results: " + compactedFiles.size());
-    // In order to be able to fail in the middle of the operation, we'll operate on lazy
-    // copies and apply the result at the end.
-    CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(false);
-    cmc.deleteResults(compactedFiles);
-    debugDumpState("Deleted compaction results");
-  }
-
   @Override
   public int getStoreCompactionPriority() {
     // If there's only L0, do what the default store does.
@@ -696,7 +660,7 @@ public class StripeStoreFileManager
       this.isFlush = isFlush;
     }
 
-    private void mergeResults(Collection<StoreFile> compactedFiles, Collection<StoreFile> results)
+    public void mergeResults(Collection<StoreFile> compactedFiles, Collection<StoreFile> results)
         throws IOException {
       assert this.compactedFiles == null && this.results == null;
       this.compactedFiles = compactedFiles;
@@ -708,20 +672,12 @@ public class StripeStoreFileManager
         processNewCandidateStripes(newStripes);
       }
       // Create new state and update parent.
-      State state = createNewState(false);
+      State state = createNewState();
       StripeStoreFileManager.this.state = state;
       updateMetadataMaps();
     }
 
-    private void deleteResults(Collection<StoreFile> compactedFiles) throws IOException {
-      this.compactedFiles = compactedFiles;
-      // Create new state and update parent.
-      State state = createNewState(true);
-      StripeStoreFileManager.this.state = state;
-      updateMetadataMaps();
-    }
-
-    private State createNewState(boolean delCompactedFiles) {
+    private State createNewState() {
       State oldState = StripeStoreFileManager.this.state;
       // Stripe count should be the same unless the end rows changed.
       assert oldState.stripeFiles.size() == this.stripeFiles.size() || this.stripeEndRows != null;
@@ -737,21 +693,9 @@ public class StripeStoreFileManager
       }
 
       List<StoreFile> newAllFiles = new ArrayList<StoreFile>(oldState.allFilesCached);
-      List<StoreFile> newAllCompactedFiles =
-          new ArrayList<StoreFile>(oldState.allCompactedFilesCached);
-      if (!isFlush) {
-        newAllFiles.removeAll(compactedFiles);
-        if (delCompactedFiles) {
-          newAllCompactedFiles.removeAll(compactedFiles);
-        } else {
-          newAllCompactedFiles.addAll(compactedFiles);
-        }
-      }
-      if (results != null) {
-        newAllFiles.addAll(results);
-      }
+      if (!isFlush) newAllFiles.removeAll(compactedFiles);
+      newAllFiles.addAll(results);
       newState.allFilesCached = ImmutableList.copyOf(newAllFiles);
-      newState.allCompactedFilesCached = ImmutableList.copyOf(newAllCompactedFiles);
       return newState;
     }
 
@@ -1002,16 +946,14 @@ public class StripeStoreFileManager
     // Order by seqnum is reversed.
     for (int i = 1; i < stripe.size(); ++i) {
       StoreFile sf = stripe.get(i);
-      synchronized (sf) {
-        long fileTs = sf.getReader().getMaxTimestamp();
-        if (fileTs < maxTs && !filesCompacting.contains(sf)) {
-          LOG.info("Found an expired store file: " + sf.getPath() + " whose maxTimeStamp is "
-              + fileTs + ", which is below " + maxTs);
-          if (expiredStoreFiles == null) {
-            expiredStoreFiles = new ArrayList<StoreFile>();
-          }
-          expiredStoreFiles.add(sf);
+      long fileTs = sf.getReader().getMaxTimestamp();
+      if (fileTs < maxTs && !filesCompacting.contains(sf)) {
+        LOG.info("Found an expired store file: " + sf.getPath()
+            + " whose maxTimeStamp is " + fileTs + ", which is below " + maxTs);
+        if (expiredStoreFiles == null) {
+          expiredStoreFiles = new ArrayList<StoreFile>();
         }
+        expiredStoreFiles.add(sf);
       }
     }
     return expiredStoreFiles;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
index 3d62c67..62e7c7c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
@@ -55,8 +55,6 @@ public class CompactionConfiguration {
   public static final String HBASE_HSTORE_COMPACTION_MIN_SIZE_KEY =
     "hbase.hstore.compaction.min.size";
   public static final String HBASE_HSTORE_COMPACTION_MAX_KEY = "hbase.hstore.compaction.max";
-  public static final String HBASE_HSTORE_COMPACTION_DISCHARGER_THREAD_COUNT =
-      "hbase.hstore.compaction.discharger.thread.count";
   public static final String HBASE_HSTORE_COMPACTION_MAX_SIZE_KEY =
     "hbase.hstore.compaction.max.size";
   public static final String HBASE_HSTORE_COMPACTION_MAX_SIZE_OFFPEAK_KEY =
@@ -66,11 +64,6 @@ public class CompactionConfiguration {
   public static final String HBASE_HSTORE_MIN_LOCALITY_TO_SKIP_MAJOR_COMPACT =
       "hbase.hstore.min.locality.to.skip.major.compact";
 
-  public static final String HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT =
-      "hbase.hfile.compaction.discharger.thread.count";
-  public static final String HBASE_HFILE_COMPACTION_DISCHARGER_INTERVAL =
-      "hbase.hfile.compaction.discharger.interval";
-
   Configuration conf;
   StoreConfigInformation storeConfigInfo;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 6f476f0..8802ea6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -109,11 +109,6 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public List<Region> getOnlineRegions() {
-    return null;
-  }
-
-  @Override
   public void addToOnlineRegions(Region r) {
     this.regions.put(r.getRegionInfo().getEncodedName(), r);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index e592536..99e364a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -197,6 +197,17 @@ public class TestIOFencing {
     }
 
     @Override
+    protected void completeCompaction(final Collection<StoreFile> compactedFiles,
+        boolean removeFiles) throws IOException {
+      try {
+        r.compactionsWaiting.countDown();
+        r.compactionsBlocked.await();
+      } catch (InterruptedException ex) {
+        throw new IOException(ex);
+      }
+      super.completeCompaction(compactedFiles, removeFiles);
+    }
+    @Override
     protected void completeCompaction(Collection<StoreFile> compactedFiles) throws IOException {
       try {
         r.compactionsWaiting.countDown();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
index a8713e7..e1304cf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.backup.example;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -45,10 +43,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
-import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -81,7 +76,6 @@ public class TestZooKeeperTableArchiveClient {
   private static ZKTableArchiveClient archivingClient;
   private final List<Path> toCleanup = new ArrayList<Path>();
   private static ClusterConnection CONNECTION;
-  private static RegionServerServices rss;
 
   /**
    * Setup the config for the cluster
@@ -96,7 +90,6 @@ public class TestZooKeeperTableArchiveClient {
     ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
     String archivingZNode = ZKTableArchiveClient.getArchiveZNode(UTIL.getConfiguration(), watcher);
     ZKUtil.createWithParents(watcher, archivingZNode);
-    rss = mock(RegionServerServices.class);
   }
 
   private static void setupConf(Configuration conf) {
@@ -176,14 +169,10 @@ public class TestZooKeeperTableArchiveClient {
 
     // create the region
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM);
-    HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
-    List<Region> regions = new ArrayList<Region>();
-    regions.add(region);
-    when(rss.getOnlineRegions()).thenReturn(regions);
-    final CompactedHFilesDischarger compactionCleaner =
-        new CompactedHFilesDischarger(100, stop, rss, false);
+    Region region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
+
     loadFlushAndCompact(region, TEST_FAM);
-    compactionCleaner.chore();
+
     // get the current hfiles in the archive directory
     List<Path> files = getAllFiles(fs, archiveDir);
     if (files == null) {
@@ -227,28 +216,18 @@ public class TestZooKeeperTableArchiveClient {
     HFileCleaner cleaner = setupAndCreateCleaner(conf, fs, archiveDir, stop);
     List<BaseHFileCleanerDelegate> cleaners = turnOnArchiving(STRING_TABLE_NAME, cleaner);
     final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0);
+
     // create the region
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM);
-    HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
-    List<Region> regions = new ArrayList<Region>();
-    regions.add(region);
-    when(rss.getOnlineRegions()).thenReturn(regions);
-    final CompactedHFilesDischarger compactionCleaner =
-        new CompactedHFilesDischarger(100, stop, rss, false);
+    Region region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
     loadFlushAndCompact(region, TEST_FAM);
-    compactionCleaner.chore();
+
     // create the another table that we don't archive
     hcd = new HColumnDescriptor(TEST_FAM);
-    HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd);
-    regions = new ArrayList<Region>();
-    regions.add(otherRegion);
-    when(rss.getOnlineRegions()).thenReturn(regions);
-    final CompactedHFilesDischarger compactionCleaner1 = new CompactedHFilesDischarger(100, stop,
-        rss, false);
+    Region otherRegion = UTIL.createTestRegion(otherTable, hcd);
     loadFlushAndCompact(otherRegion, TEST_FAM);
-    compactionCleaner1.chore();
+
     // get the current hfiles in the archive directory
-    // Should  be archived
     List<Path> files = getAllFiles(fs, archiveDir);
     if (files == null) {
       FSUtils.logFileSystemState(fs, archiveDir, LOG);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
index 07ca2b9..13fcb24 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
@@ -74,7 +74,7 @@ public class TestHeapSize  {
     LOG.info("name=" + b.getName()); 
     LOG.info("specname=" + b.getSpecName()); 
     LOG.info("specvendor=" + b.getSpecVendor()); 
-    LOG.info("vmname=" + b.getVmName());
+    LOG.info("vmname=" + b.getVmName()); 
     LOG.info("vmversion=" + b.getVmVersion()); 
     LOG.info("vmvendor=" + b.getVmVendor()); 
     Map<String, String> p = b.getSystemProperties();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index bc3da47..bb1a073 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -29,7 +29,6 @@ import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
@@ -75,7 +74,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -96,8 +94,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 import org.mockito.Mockito;
 
-import com.google.common.collect.Lists;
-
 /**
  * Simple test for {@link CellSortReducer} and {@link HFileOutputFormat2}.
  * Sets up and runs a mapreduce job that writes hfile output.
@@ -1000,12 +996,6 @@ public class TestHFileOutputFormat2  {
         quickPoll(new Callable<Boolean>() {
           @Override
           public Boolean call() throws Exception {
-            List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAME);
-            for (HRegion region : regions) {
-              for (Store store : region.getStores()) {
-                store.closeAndArchiveCompactedFiles();
-              }
-            }
             return fs.listStatus(storePath).length == 1;
           }
         }, 5000);
@@ -1019,12 +1009,6 @@ public class TestHFileOutputFormat2  {
       quickPoll(new Callable<Boolean>() {
         @Override
         public Boolean call() throws Exception {
-          List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAME);
-          for (HRegion region : regions) {
-            for (Store store : region.getStores()) {
-              store.closeAndArchiveCompactedFiles();
-            }
-          }
           return fs.listStatus(storePath).length == 1;
         }
       }, 5000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index e227866..8aa38d6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -461,11 +461,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public List<Region> getOnlineRegions() {
-    return null;
-  }
-
-  @Override
   public OpenRegionResponse openRegion(RpcController controller,
       OpenRegionRequest request) throws ServiceException {
     // TODO Auto-generated method stub

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
index 42be207..08bafdc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
@@ -47,10 +47,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnaps
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
-import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
@@ -58,7 +56,6 @@ import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -123,7 +120,6 @@ public class TestSnapshotFromMaster {
     conf.setLong(SnapshotHFileCleaner.HFILE_CACHE_REFRESH_PERIOD_CONF_KEY, cacheRefreshPeriod);
     conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
       ConstantSizeRegionSplitPolicy.class.getName());
-    conf.setInt("hbase.hfile.compactions.cleaner.interval", 20 * 1000);
 
   }
 
@@ -324,17 +320,6 @@ public class TestSnapshotFromMaster {
       region.waitForFlushesAndCompactions(); // enable can trigger a compaction, wait for it.
       region.compactStores(); // min is 2 so will compact and archive
     }
-    List<RegionServerThread> regionServerThreads = UTIL.getMiniHBaseCluster()
-        .getRegionServerThreads();
-    HRegionServer hrs = null;
-    for (RegionServerThread rs : regionServerThreads) {
-      if (!rs.getRegionServer().getOnlineRegions(TABLE_NAME).isEmpty()) {
-        hrs = rs.getRegionServer();
-        break;
-      }
-    }
-    CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, hrs, false);
-    cleaner.chore();
     LOG.info("After compaction File-System state");
     FSUtils.logFileSystemState(fs, rootDir, LOG);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5228a0b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
index 76e496d..3a12674 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
@@ -79,11 +79,6 @@ public class MockStoreFile extends StoreFile {
   }
 
   @Override
-  public boolean isCompactedAway() {
-    return false;
-  }
-
-  @Override
   public byte[] getMetadataValue(byte[] key) {
     return this.metadata.get(key);
   }