HBASE-14970 Backport HBASE-13082 and its sub-jira to branch-1 (Ram)
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/58521869 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/58521869 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/58521869 Branch: refs/heads/branch-1 Commit: 58521869b06a63894e422e9c9403e48b4b12f388 Parents: cea4378 Author: ramkrishna <ramkrishna.s.vasude...@gmail.com> Authored: Thu Jan 21 21:22:40 2016 +0530 Committer: ramkrishna <ramkrishna.s.vasude...@gmail.com> Committed: Thu Jan 21 21:22:40 2016 +0530 ---------------------------------------------------------------------- .../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, 1326 insertions(+), 180 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 ac76edb..a7759c5 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,7 +265,14 @@ public enum EventType { * * RS_REGION_REPLICA_FLUSH */ - RS_REGION_REPLICA_FLUSH (82, ExecutorType.RS_REGION_REPLICA_FLUSH_OPS); + 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); private final int code; private final ExecutorType executor; http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 d0f6bee..5a16149 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,7 +46,8 @@ public enum ExecutorType { RS_CLOSE_META (25), RS_PARALLEL_SEEK (26), RS_LOG_REPLAY_OPS (27), - RS_REGION_REPLICA_FLUSH_OPS (28); + RS_REGION_REPLICA_FLUSH_OPS (28), + RS_COMPACTED_FILES_DISCHARGER (29); ExecutorType(int value) {} http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 42cca2b..018e173 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,6 +38,7 @@ 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; @@ -86,7 +87,8 @@ public class ExecutorService { * started with the same name, this throws a RuntimeException. * @param name Name of the service to start. */ - void startExecutorService(String name, int maxThreads) { + @VisibleForTesting + public 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/58521869/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 36b7559..0bc75e7 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,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -33,5 +34,5 @@ public interface ChangedReadersObserver { * Notify observers. * @throws IOException e */ - void updateReaders() throws IOException; + void updateReaders(List<StoreFile> sfs) throws IOException; } http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 new file mode 100644 index 0000000..02160d8 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java @@ -0,0 +1,45 @@ +/** + * 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/58521869/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 new file mode 100644 index 0000000..9f6c65c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java @@ -0,0 +1,110 @@ +/** + * 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/58521869/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 6000f01..6611202 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,6 +54,13 @@ 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) { @@ -75,6 +82,11 @@ 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); @@ -89,19 +101,55 @@ 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> compactedFiles, Collection<StoreFile> results) { + Collection<StoreFile> newCompactedfiles, Collection<StoreFile> results) { ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles); - newStoreFiles.removeAll(compactedFiles); + newStoreFiles.removeAll(newCompactedfiles); 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 @@ -166,6 +214,12 @@ 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/58521869/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 912ff50..5ad8fa3 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 @@ -1517,6 +1517,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (this.metricsRegionWrapper != null) { Closeables.closeQuietly(this.metricsRegionWrapper); } + status.markComplete("Closed"); LOG.info("Closed " + this); return result; @@ -6790,6 +6791,10 @@ 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/58521869/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 7351fa8..7287f78 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,6 +133,7 @@ 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; @@ -479,6 +480,8 @@ public class HRegionServer extends HasThread implements */ protected final ConfigurationManager configurationManager; + private CompactedHFilesDischarger compactedFileDischarger; + /** * Starts a HRegionServer at the default location. * @param conf @@ -611,6 +614,16 @@ 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); } /* @@ -1708,7 +1721,9 @@ 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", @@ -2732,6 +2747,15 @@ 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/58521869/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 a19407a..cb62c95 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,7 +56,6 @@ 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; @@ -273,7 +272,6 @@ 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) { @@ -531,14 +529,15 @@ public class HStore implements Store { try { Future<StoreFile> future = completionService.take(); StoreFile storeFile = future.get(); - long length = storeFile.getReader().length(); - this.storeSize += length; - this.totalUncompressedBytes += - storeFile.getReader().getTotalUncompressedBytes(); - if (LOG.isDebugEnabled()) { - LOG.debug("loaded " + storeFile.toStringDetailed()); + 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); } - results.add(storeFile); } catch (InterruptedException e) { if (ioe == null) ioe = new InterruptedIOException(e.getMessage()); } catch (ExecutionException e) { @@ -636,8 +635,7 @@ public class HStore implements Store { region.getMVCC().advanceTo(this.getMaxSequenceId()); } - // notify scanners, close file readers, and recompute store size - completeCompaction(toBeRemovedStoreFiles, false); + completeCompaction(toBeRemovedStoreFiles); } private StoreFile createStoreFileAndReader(final Path p) throws IOException { @@ -814,7 +812,6 @@ 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 [" @@ -830,7 +827,12 @@ 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 @@ -845,7 +847,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; } @@ -1067,10 +1069,8 @@ public class HStore implements Store { // the lock. this.lock.writeLock().unlock(); } - - // Tell listeners of the change in readers. - notifyChangedReadersObservers(); - + // notify to be called here - only in case of flushes + notifyChangedReadersObservers(sfs); 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() throws IOException { - for (ChangedReadersObserver o: this.changedReaderObservers) { - o.updateReaders(); + private void notifyChangedReadersObservers(List<StoreFile> sfs) throws IOException { + for (ChangedReadersObserver o : this.changedReaderObservers) { + o.updateReaders(sfs); } } @@ -1130,6 +1130,30 @@ 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); } @@ -1248,7 +1272,7 @@ public class HStore implements Store { compactedCellsSize += getCompactionProgress().totalCompactedSize; } // At this point the store will use new files for all new scanners. - completeCompaction(filesToCompact, true); // Archive old files & update store size. + completeCompaction(filesToCompact); // update store size. logCompactionEndMessage(cr, sfs, compactionStartTime); return sfs; @@ -1436,7 +1460,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, removeFiles); + this.completeCompaction(inputStoreFiles); } } @@ -1488,7 +1512,7 @@ public class HStore implements Store { this.getCoprocessorHost().postCompact(this, sf, null); } replaceStoreFiles(filesToCompact, Lists.newArrayList(sf)); - completeCompaction(filesToCompact, true); + completeCompaction(filesToCompact); } } finally { synchronized (filesCompacting) { @@ -1771,32 +1795,7 @@ public class HStore implements Store { @VisibleForTesting protected void completeCompaction(final Collection<StoreFile> compactedFiles, boolean removeFiles) throws IOException { - 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 + LOG.debug("Completing compaction..."); this.storeSize = 0L; this.totalUncompressedBytes = 0L; for (StoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) { @@ -2490,4 +2489,92 @@ 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/58521869/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 60fc9fb..310108c 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,4 +67,10 @@ 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/58521869/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 e319f90..0f12b0a 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,24 +124,15 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner { @Override public boolean seekToPreviousRow(Cell key) throws IOException { - lock.lock(); - try { - checkReseek(); - return this.heap.seekToPreviousRow(key); - } finally { - lock.unlock(); - } - + boolean flushed = checkFlushed(); + checkReseek(flushed); + return this.heap.seekToPreviousRow(key); } @Override public boolean backwardSeek(Cell key) throws IOException { - lock.lock(); - try { - checkReseek(); - return this.heap.backwardSeek(key); - } finally { - lock.unlock(); - } + boolean flushed = checkFlushed(); + checkReseek(flushed); + return this.heap.backwardSeek(key); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 9f17526..ddcd4e9 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,6 +105,25 @@ 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(); @@ -480,4 +499,9 @@ 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/58521869/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 46a64f2..57a272e 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,6 +30,8 @@ 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; @@ -61,6 +63,7 @@ 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; @@ -368,6 +371,19 @@ 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. */ @@ -537,6 +553,15 @@ 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 */ @@ -1072,6 +1097,12 @@ 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 { @@ -1079,6 +1110,10 @@ 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); @@ -1130,12 +1165,36 @@ 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. @@ -1620,7 +1679,13 @@ public class StoreFile { private static class GetFileSize implements Function<StoreFile, Long> { @Override public Long apply(StoreFile sf) { - return sf.getReader().length(); + 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; + } } } http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 11993db..7e70547 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 compaction results into the structure. + * Adds only the new compaction results into the structure. * @param compactedFiles The input files for the compaction. * @param results The resulting files for the compaction. */ @@ -61,12 +61,26 @@ 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. @@ -74,6 +88,15 @@ 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/58521869/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 1d2f7e5..4055188 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,6 +51,7 @@ 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; @@ -171,7 +172,7 @@ public class StoreFileScanner implements KeyValueScanner { try { try { if(!seekAtOrAfter(hfs, key)) { - close(); + this.cur = null; return false; } @@ -198,7 +199,7 @@ public class StoreFileScanner implements KeyValueScanner { try { try { if (!reseekAtOrAfter(hfs, key)) { - close(); + this.cur = null; return false; } setCurrentCell(hfs.getKeyValue()); @@ -244,7 +245,6 @@ public class StoreFileScanner implements KeyValueScanner { } if (cur == null) { - close(); return false; } @@ -252,8 +252,12 @@ 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; } /** @@ -454,7 +458,7 @@ public class StoreFileScanner implements KeyValueScanner { if (seekCount != null) seekCount.incrementAndGet(); if (!hfs.seekBefore(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey.getKeyLength())) { - close(); + this.cur = null; return false; } KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getKeyValue() http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 f1735eb..bcb866a 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,7 +121,14 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // A flag whether use pread for scan private boolean scanUsePread = false; - protected ReentrantLock lock = new ReentrantLock(); + // 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(); private final long readPt; @@ -166,6 +173,9 @@ 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. @@ -203,7 +213,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()); } @@ -260,7 +270,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()); } @@ -299,6 +309,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner } // Seek all scanners to the initial key seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled); + addCurrentScanners(scanners); resetKVHeap(scanners, scanInfo.getComparator()); } @@ -392,6 +403,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner if (kvs.shouldUseScanner(scan, store, expiredTimestampCutoff)) { scanners.add(kvs); + } else { + kvs.close(); } } return scanners; @@ -399,15 +412,10 @@ 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 @@ -418,8 +426,6 @@ 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 @@ -429,21 +435,14 @@ 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 { - lock.lock(); - try { + boolean flushed = checkFlushed(); // reset matcher state, in case that underlying store changed - checkReseek(); + checkReseek(flushed); return this.heap.seek(key); - } finally { - lock.unlock(); - } } @Override @@ -459,13 +458,11 @@ 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"); } - if (checkReseek()) { + boolean flushed = checkFlushed(); + if (checkReseek(flushed)) { return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues(); } @@ -643,9 +640,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // No more keys close(); return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); - } finally { - lock.unlock(); - } } /* @@ -682,9 +676,18 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // Implementation of ChangedReadersObserver @Override - public void updateReaders() throws IOException { - lock.lock(); + public void updateReaders(List<StoreFile> sfs) throws IOException { + flushed = true; + flushLock.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 @@ -695,7 +698,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner if (this.heap == null) return; // this could be null. - this.lastTop = this.peek(); + this.lastTop = this.heap.peek(); //DebugPrint.println("SS updateReaders, topKey = " + lastTop); @@ -704,18 +707,16 @@ 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() throws IOException { - if (this.heap == null && this.lastTop != null) { + protected boolean checkReseek(boolean flushed) throws IOException { + if (flushed && this.lastTop != null) { resetScannerStack(this.lastTop); if (this.heap.peek() == null || store.getComparator().compareRows(this.lastTop, this.heap.peek()) != 0) { @@ -731,21 +732,37 @@ 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 */ - List<KeyValueScanner> scanners = getScannersNoCompaction(); + * could have done it now by storing the scan object from the constructor + */ - // Seek all scanners to the initial key - seekScanners(scanners, lastTopKey, false, parallelSeekEnabled); + 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(); + } + // Seek the new scanners to the last 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(scanners, store.getComparator()); - + resetKVHeap(this.currentScanners, 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. @@ -796,19 +813,36 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner @Override public boolean reseek(Cell kv) throws IOException { - 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(); + 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); if (explicitColumnQuery && lazySeekEnabledGlobally) { return heap.requestSeek(kv, true, useRowColBloom); } return heap.reseek(kv); - } finally { - lock.unlock(); + } + + 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; } + return false; } @Override http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 1bbaefe..5b79790 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,6 +104,7 @@ 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; @@ -139,8 +140,14 @@ 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"); } @@ -155,6 +162,13 @@ 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(); } @@ -304,9 +318,31 @@ 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. @@ -660,7 +696,7 @@ public class StripeStoreFileManager this.isFlush = isFlush; } - public void mergeResults(Collection<StoreFile> compactedFiles, Collection<StoreFile> results) + private void mergeResults(Collection<StoreFile> compactedFiles, Collection<StoreFile> results) throws IOException { assert this.compactedFiles == null && this.results == null; this.compactedFiles = compactedFiles; @@ -672,12 +708,20 @@ public class StripeStoreFileManager processNewCandidateStripes(newStripes); } // Create new state and update parent. - State state = createNewState(); + State state = createNewState(false); StripeStoreFileManager.this.state = state; updateMetadataMaps(); } - private State createNewState() { + 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) { 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; @@ -693,9 +737,21 @@ public class StripeStoreFileManager } List<StoreFile> newAllFiles = new ArrayList<StoreFile>(oldState.allFilesCached); - if (!isFlush) newAllFiles.removeAll(compactedFiles); - newAllFiles.addAll(results); + 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); + } newState.allFilesCached = ImmutableList.copyOf(newAllFiles); + newState.allCompactedFilesCached = ImmutableList.copyOf(newAllCompactedFiles); return newState; } @@ -946,14 +1002,16 @@ public class StripeStoreFileManager // Order by seqnum is reversed. for (int i = 1; i < stripe.size(); ++i) { StoreFile sf = stripe.get(i); - 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>(); + 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); } - expiredStoreFiles.add(sf); } } return expiredStoreFiles; http://git-wip-us.apache.org/repos/asf/hbase/blob/58521869/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 62e7c7c..3d62c67 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,6 +55,8 @@ 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 = @@ -64,6 +66,11 @@ 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/58521869/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 8802ea6..6f476f0 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,6 +109,11 @@ 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/58521869/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 99e364a..e592536 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,17 +197,6 @@ 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/58521869/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 e1304cf..a8713e7 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,6 +20,8 @@ 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; @@ -43,7 +45,10 @@ 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; @@ -76,6 +81,7 @@ 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 @@ -90,6 +96,7 @@ 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) { @@ -169,10 +176,14 @@ public class TestZooKeeperTableArchiveClient { // create the region HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM); - Region region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); - + 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); loadFlushAndCompact(region, TEST_FAM); - + compactionCleaner.chore(); // get the current hfiles in the archive directory List<Path> files = getAllFiles(fs, archiveDir); if (files == null) { @@ -216,18 +227,28 @@ 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); - Region region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); + 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); loadFlushAndCompact(region, TEST_FAM); - + compactionCleaner.chore(); // create the another table that we don't archive hcd = new HColumnDescriptor(TEST_FAM); - Region otherRegion = UTIL.createTestRegion(otherTable, hcd); + 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); 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/58521869/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 13fcb24..07ca2b9 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/58521869/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 bb1a073..bc3da47 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,6 +29,7 @@ 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; @@ -74,6 +75,7 @@ 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; @@ -94,6 +96,8 @@ 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. @@ -996,6 +1000,12 @@ 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); @@ -1009,6 +1019,12 @@ 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/58521869/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 8aa38d6..e227866 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,6 +461,11 @@ 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/58521869/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 08bafdc..42be207 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,8 +47,10 @@ 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; @@ -56,6 +58,7 @@ 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; @@ -120,6 +123,7 @@ 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); } @@ -320,6 +324,17 @@ 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/58521869/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 3a12674..76e496d 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,6 +79,11 @@ public class MockStoreFile extends StoreFile { } @Override + public boolean isCompactedAway() { + return false; + } + + @Override public byte[] getMetadataValue(byte[] key) { return this.metadata.get(key); }