From 13b69a935309a68747f39b0204e7d5a17851244d Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 29 Jul 2021 18:35:19 +0800 Subject: [PATCH 01/26] HBASE-26064 Introduce a StoreFileTracker to abstract the store file tracking logic Signed-off-by: Wellington Chevreuil --- .../hbase/mob/DefaultMobStoreCompactor.java | 22 +- .../hbase/mob/DefaultMobStoreFlusher.java | 4 +- .../CreateStoreFileWriterParams.java | 134 ++++ .../regionserver/DateTieredStoreEngine.java | 5 +- .../regionserver/DefaultStoreEngine.java | 5 +- .../regionserver/DefaultStoreFlusher.java | 11 +- .../hadoop/hbase/regionserver/HMobStore.java | 3 +- .../hbase/regionserver/HRegionFileSystem.java | 10 +- .../hadoop/hbase/regionserver/HStore.java | 729 ++++-------------- .../hbase/regionserver/StoreContext.java | 9 + .../hbase/regionserver/StoreEngine.java | 461 ++++++++++- .../hbase/regionserver/StoreFileManager.java | 9 + .../hbase/regionserver/StoreFlusher.java | 9 +- .../hadoop/hbase/regionserver/StoreUtils.java | 37 +- .../hbase/regionserver/StripeStoreEngine.java | 9 +- .../regionserver/StripeStoreFlusher.java | 9 +- .../AbstractMultiOutputCompactor.java | 7 +- .../regionserver/compactions/Compactor.java | 36 +- .../compactions/DefaultCompactor.java | 16 +- .../DefaultStoreFileTracker.java | 61 ++ .../storefiletracker/StoreFileTracker.java | 75 ++ .../StoreFileTrackerBase.java | 178 +++++ .../StoreFileTrackerFactory.java | 35 + .../compaction/MajorCompactionRequest.java | 1 - .../apache/hadoop/hbase/TestIOFencing.java | 12 +- .../TestCacheOnWriteInSchema.java | 6 +- .../regionserver/TestDefaultStoreEngine.java | 5 +- .../hbase/regionserver/TestHRegion.java | 4 +- .../hadoop/hbase/regionserver/TestHStore.java | 33 +- .../TestRegionMergeTransactionOnCluster.java | 6 +- .../TestStoreFileRefresherChore.java | 3 +- .../regionserver/TestStoreScannerClosure.java | 6 +- .../regionserver/TestStripeStoreEngine.java | 2 + .../compactions/TestDateTieredCompactor.java | 12 +- .../TestStripeCompactionPolicy.java | 12 +- .../compactions/TestStripeCompactor.java | 12 +- 36 files changed, 1261 insertions(+), 727 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CreateStoreFileWriterParams.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index c45fdff7ca16..01fe0005f048 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -29,7 +29,6 @@ import java.util.List; import java.util.Map.Entry; import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -144,17 +143,16 @@ public InternalScanner createScanner(ScanInfo scanInfo, List s }; private final CellSinkFactory writerFactory = - new CellSinkFactory() { - @Override - public StoreFileWriter createWriter(InternalScanner scanner, - org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd, - boolean shouldDropBehind, boolean major) throws IOException { - // make this writer with tags always because of possible new cells with tags. - return store.createWriterInTmp(fd.maxKeyCount, - major ? majorCompactionCompression : minorCompactionCompression, - true, true, true, shouldDropBehind); - } - }; + new CellSinkFactory() { + @Override + public StoreFileWriter createWriter(InternalScanner scanner, + org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd, + boolean shouldDropBehind, boolean major) throws IOException { + // make this writer with tags always because of possible new cells with tags. + return store.getStoreEngine().createWriter( + createParams(fd, shouldDropBehind, major).includeMVCCReadpoint(true).includesTag(true)); + } + }; public DefaultMobStoreCompactor(Configuration conf, HStore store) { super(conf, store); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index 480b85c58dfe..4a1dc7b33a51 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -25,7 +25,6 @@ import java.util.HashSet; import java.util.List; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -127,8 +126,7 @@ public List flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId, synchronized (flushLock) { status.setStatus("Flushing " + store + ": creating writer"); // Write the map out to the disk - writer = store.createWriterInTmp(cellsCount, store.getColumnFamilyDescriptor().getCompressionType(), - false, true, true, false); + writer = createWriter(snapshot, true); IOException e = null; try { // It's a mob store, flush the cells in a mob way. This is the difference of flushing diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CreateStoreFileWriterParams.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CreateStoreFileWriterParams.java new file mode 100644 index 000000000000..10cd9f009e4a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CreateStoreFileWriterParams.java @@ -0,0 +1,134 @@ +/** + * 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 org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public final class CreateStoreFileWriterParams { + + private long maxKeyCount; + + private Compression.Algorithm compression; + + private boolean isCompaction; + + private boolean includeMVCCReadpoint; + + private boolean includesTag; + + private boolean shouldDropBehind; + + private long totalCompactedFilesSize = -1; + + private String fileStoragePolicy = HConstants.EMPTY_STRING; + + private CreateStoreFileWriterParams() { + } + + public long maxKeyCount() { + return maxKeyCount; + } + + public CreateStoreFileWriterParams maxKeyCount(long maxKeyCount) { + this.maxKeyCount = maxKeyCount; + return this; + } + + public Compression.Algorithm compression() { + return compression; + } + + /** + * Set the compression algorithm to use + */ + public CreateStoreFileWriterParams compression(Compression.Algorithm compression) { + this.compression = compression; + return this; + } + + public boolean isCompaction() { + return isCompaction; + } + + /** + * Whether we are creating a new file in a compaction + */ + public CreateStoreFileWriterParams isCompaction(boolean isCompaction) { + this.isCompaction = isCompaction; + return this; + } + + public boolean includeMVCCReadpoint() { + return includeMVCCReadpoint; + } + + /** + * Whether to include MVCC or not + */ + public CreateStoreFileWriterParams includeMVCCReadpoint(boolean includeMVCCReadpoint) { + this.includeMVCCReadpoint = includeMVCCReadpoint; + return this; + } + + public boolean includesTag() { + return includesTag; + } + + /** + * Whether to includesTag or not + */ + public CreateStoreFileWriterParams includesTag(boolean includesTag) { + this.includesTag = includesTag; + return this; + } + + public boolean shouldDropBehind() { + return shouldDropBehind; + } + + public CreateStoreFileWriterParams shouldDropBehind(boolean shouldDropBehind) { + this.shouldDropBehind = shouldDropBehind; + return this; + } + + public long totalCompactedFilesSize() { + return totalCompactedFilesSize; + } + + public CreateStoreFileWriterParams totalCompactedFilesSize(long totalCompactedFilesSize) { + this.totalCompactedFilesSize = totalCompactedFilesSize; + return this; + } + + public String fileStoragePolicy() { + return fileStoragePolicy; + } + + public CreateStoreFileWriterParams fileStoragePolicy(String fileStoragePolicy) { + this.fileStoragePolicy = fileStoragePolicy; + return this; + } + + public static CreateStoreFileWriterParams create() { + return new CreateStoreFileWriterParams(); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java index 1df953d93c96..7422d9112eab 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java @@ -19,18 +19,17 @@ import java.io.IOException; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; +import org.apache.yetus.audience.InterfaceAudience; /** * HBASE-15400 This store engine allows us to store data in date tiered layout with exponential diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java index 58f8bbbb6ac5..693b9c93b9fd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; @@ -39,8 +38,8 @@ * their derivatives. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) -public class DefaultStoreEngine extends StoreEngine< - DefaultStoreFlusher, RatioBasedCompactionPolicy, DefaultCompactor, DefaultStoreFileManager> { +public class DefaultStoreEngine extends StoreEngine { public static final String DEFAULT_STORE_FLUSHER_CLASS_KEY = "hbase.hstore.defaultengine.storeflusher.class"; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java index a7d7fb1f3d56..306760d7ce6a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java @@ -21,15 +21,14 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; - -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.util.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Default implementation of StoreFlusher. @@ -60,9 +59,7 @@ public List flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId, synchronized (flushLock) { status.setStatus("Flushing " + store + ": creating writer"); // Write the map out to the disk - writer = store.createWriterInTmp(cellsCount, - store.getColumnFamilyDescriptor().getCompressionType(), false, true, - snapshot.isTagsPresent(), false); + writer = createWriter(snapshot, false); IOException e = null; try { performFlush(scanner, writer, throughputController); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java index 7ce7f0310c7d..b00a50c522fc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java @@ -28,7 +28,6 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -158,7 +157,7 @@ protected KeyValueScanner createScanner(Scan scan, ScanInfo scanInfo, protected StoreEngine createStoreEngine(HStore store, Configuration conf, CellComparator cellComparator) throws IOException { MobStoreEngine engine = new MobStoreEngine(); - engine.createComponents(conf, store, cellComparator); + engine.createComponentsOnce(conf, store, cellComparator); return engine; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index 667eabfcd281..2f5f8d7e34cd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -145,7 +145,7 @@ public Path getRegionDir() { // Temp Helpers // =========================================================================== /** @return {@link Path} to the region's temp directory, used for file creations */ - Path getTempDir() { + public Path getTempDir() { return new Path(getRegionDir(), REGION_TEMP_DIR); } @@ -240,11 +240,7 @@ public String getStoragePolicyName(String familyName) { * @param familyName Column Family Name * @return a set of {@link StoreFileInfo} for the specified family. */ - public Collection getStoreFiles(final byte[] familyName) throws IOException { - return getStoreFiles(Bytes.toString(familyName)); - } - - public Collection getStoreFiles(final String familyName) throws IOException { + public List getStoreFiles(final String familyName) throws IOException { return getStoreFiles(familyName, true); } @@ -254,7 +250,7 @@ public Collection getStoreFiles(final String familyName) throws I * @param familyName Column Family Name * @return a set of {@link StoreFileInfo} for the specified family. */ - public Collection getStoreFiles(final String familyName, final boolean validate) + public List getStoreFiles(final String familyName, final boolean validate) throws IOException { Path familyDir = getStoreDir(familyName); FileStatus[] files = CommonFSUtils.listStatus(this.fs, familyDir); 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 0ee7b5777fc3..20bf7c6bc6dc 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver; +import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; import java.io.InterruptedIOException; import java.net.InetSocketAddress; @@ -47,8 +48,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.ReentrantLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.Predicate; import java.util.function.ToLongFunction; import java.util.stream.Collectors; import java.util.stream.LongStream; @@ -70,17 +69,12 @@ import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver; import org.apache.hadoop.hbase.coprocessor.ReadOnlyConfiguration; import org.apache.hadoop.hbase.io.HeapSize; -import org.apache.hadoop.hbase.io.compress.Compression; -import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.io.hfile.HFileContext; -import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.io.hfile.InvalidHFileException; -import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.quotas.RegionSizeStore; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; @@ -110,7 +104,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; import org.apache.hbase.thirdparty.org.apache.commons.collections4.IterableUtils; @@ -165,16 +158,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, private boolean cacheOnWriteLogged; - /** - * RWLock for store operations. - * Locked in shared mode when the list of component stores is looked at: - * - all reads/writes to table data - * - checking for split - * Locked in exclusive mode when the list of component stores is modified: - * - closing - * - completing a compaction - */ - final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); /** * Lock specific to archiving compacted store files. This avoids races around * the combination of retrieving the list of compacted files and moving them to @@ -283,14 +266,8 @@ protected HStore(final HRegion region, final ColumnFamilyDescriptor family, } this.storeEngine = createStoreEngine(this, this.conf, region.getCellComparator()); - List hStoreFiles = loadStoreFiles(warmup); - // Move the storeSize calculation out of loadStoreFiles() method, because the secondary read - // replica's refreshStoreFiles() will also use loadStoreFiles() to refresh its store files and - // update the storeSize in the refreshStoreSizeAndTotalBytes() finally (just like compaction) , so - // no need calculate the storeSize twice. - this.storeSize.addAndGet(getStorefilesSize(hStoreFiles, sf -> true)); - this.totalUncompressedBytes.addAndGet(getTotalUncompressedBytes(hStoreFiles)); - this.storeEngine.getStoreFileManager().loadFiles(hStoreFiles); + storeEngine.initialize(warmup); + refreshStoreSizeAndTotalBytes(); flushRetriesNumber = conf.getInt( "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER); @@ -510,105 +487,18 @@ void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) { this.dataBlockEncoder = blockEncoder; } - /** - * Creates an unsorted list of StoreFile loaded in parallel - * from the given directory. - */ - private List loadStoreFiles(boolean warmup) throws IOException { - Collection files = getRegionFileSystem().getStoreFiles(getColumnFamilyName()); - return openStoreFiles(files, warmup); - } - - private List openStoreFiles(Collection files, boolean warmup) - throws IOException { - if (CollectionUtils.isEmpty(files)) { - return Collections.emptyList(); - } - // initialize the thread pool for opening store files in parallel.. - ThreadPoolExecutor storeFileOpenerThreadPool = - this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpener-" - + this.region.getRegionInfo().getEncodedName() + "-" + this.getColumnFamilyName()); - CompletionService completionService = - new ExecutorCompletionService<>(storeFileOpenerThreadPool); - - int totalValidStoreFile = 0; - for (StoreFileInfo storeFileInfo : files) { - // The StoreFileInfo will carry store configuration down to HFile, we need to set it to - // our store's CompoundConfiguration here. - storeFileInfo.setConf(conf); - // open each store file in parallel - completionService.submit(() -> this.createStoreFileAndReader(storeFileInfo)); - totalValidStoreFile++; - } - - Set compactedStoreFiles = new HashSet<>(); - ArrayList results = new ArrayList<>(files.size()); - IOException ioe = null; - try { - for (int i = 0; i < totalValidStoreFile; i++) { - try { - HStoreFile storeFile = completionService.take().get(); - if (storeFile != null) { - LOG.debug("loaded {}", storeFile); - results.add(storeFile); - compactedStoreFiles.addAll(storeFile.getCompactedStoreFiles()); - } - } catch (InterruptedException e) { - if (ioe == null) { - ioe = new InterruptedIOException(e.getMessage()); - } - } catch (ExecutionException e) { - if (ioe == null) { - ioe = new IOException(e.getCause()); - } - } - } - } finally { - storeFileOpenerThreadPool.shutdownNow(); - } - if (ioe != null) { - // close StoreFile readers - boolean evictOnClose = - getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true; - for (HStoreFile file : results) { - try { - if (file != null) { - file.closeStoreFile(evictOnClose); - } - } catch (IOException e) { - LOG.warn("Could not close store file {}", file, e); - } - } - throw ioe; - } - - // Should not archive the compacted store files when region warmup. See HBASE-22163. - if (!warmup) { - // Remove the compacted files from result - List filesToRemove = new ArrayList<>(compactedStoreFiles.size()); - for (HStoreFile storeFile : results) { - if (compactedStoreFiles.contains(storeFile.getPath().getName())) { - LOG.warn("Clearing the compacted storefile {} from {}", storeFile, this); - storeFile.getReader().close(storeFile.getCacheConf() != null ? - storeFile.getCacheConf().shouldEvictOnClose() : true); - filesToRemove.add(storeFile); - } - } - results.removeAll(filesToRemove); - if (!filesToRemove.isEmpty() && this.isPrimaryReplicaStore()) { - LOG.debug("Moving the files {} to archive", filesToRemove); - getRegionFileSystem().removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(), - filesToRemove); - } - } - - return results; + private void postRefreshStoreFiles() throws IOException { + // Advance the memstore read point to be at least the new store files seqIds so that + // readers might pick it up. This assumes that the store is not getting any writes (otherwise + // in-flight transactions might be made visible) + getMaxSequenceId().ifPresent(region.getMVCC()::advanceTo); + refreshStoreSizeAndTotalBytes(); } @Override public void refreshStoreFiles() throws IOException { - Collection newFiles = getRegionFileSystem().getStoreFiles(getColumnFamilyName()); - refreshStoreFilesInternal(newFiles); + storeEngine.refreshStoreFiles(); + postRefreshStoreFiles(); } /** @@ -616,89 +506,8 @@ public void refreshStoreFiles() throws IOException { * region replicas to keep up to date with the primary region files. */ public void refreshStoreFiles(Collection newFiles) throws IOException { - List storeFiles = new ArrayList<>(newFiles.size()); - for (String file : newFiles) { - storeFiles.add(getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), file)); - } - refreshStoreFilesInternal(storeFiles); - } - - /** - * Checks the underlying store files, and opens the files that have not - * been opened, and removes the store file readers for store files no longer - * available. Mainly used by secondary region replicas to keep up to date with - * the primary region files. - */ - private void refreshStoreFilesInternal(Collection newFiles) throws IOException { - StoreFileManager sfm = storeEngine.getStoreFileManager(); - Collection currentFiles = sfm.getStorefiles(); - Collection compactedFiles = sfm.getCompactedfiles(); - if (currentFiles == null) { - currentFiles = Collections.emptySet(); - } - if (newFiles == null) { - newFiles = Collections.emptySet(); - } - if (compactedFiles == null) { - compactedFiles = Collections.emptySet(); - } - - HashMap currentFilesSet = new HashMap<>(currentFiles.size()); - for (HStoreFile sf : currentFiles) { - currentFilesSet.put(sf.getFileInfo(), sf); - } - HashMap compactedFilesSet = new HashMap<>(compactedFiles.size()); - for (HStoreFile sf : compactedFiles) { - compactedFilesSet.put(sf.getFileInfo(), sf); - } - - Set newFilesSet = new HashSet(newFiles); - // Exclude the files that have already been compacted - newFilesSet = Sets.difference(newFilesSet, compactedFilesSet.keySet()); - Set toBeAddedFiles = Sets.difference(newFilesSet, currentFilesSet.keySet()); - Set toBeRemovedFiles = Sets.difference(currentFilesSet.keySet(), newFilesSet); - - if (toBeAddedFiles.isEmpty() && toBeRemovedFiles.isEmpty()) { - return; - } - - LOG.info("Refreshing store files for " + this + " files to add: " - + toBeAddedFiles + " files to remove: " + toBeRemovedFiles); - - Set toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size()); - for (StoreFileInfo sfi : toBeRemovedFiles) { - toBeRemovedStoreFiles.add(currentFilesSet.get(sfi)); - } - - // try to open the files - List openedFiles = openStoreFiles(toBeAddedFiles, false); - - // propogate the file changes to the underlying store file manager - replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); //won't throw an exception - - // Advance the memstore read point to be at least the new store files seqIds so that - // readers might pick it up. This assumes that the store is not getting any writes (otherwise - // in-flight transactions might be made visible) - if (!toBeAddedFiles.isEmpty()) { - // we must have the max sequence id here as we do have several store files - region.getMVCC().advanceTo(this.getMaxSequenceId().getAsLong()); - } - - refreshStoreSizeAndTotalBytes(); - } - - protected HStoreFile createStoreFileAndReader(final Path p) throws IOException { - StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(), - p, isPrimaryReplicaStore()); - return createStoreFileAndReader(info); - } - - private HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException { - info.setRegionCoprocessorHost(this.region.getCoprocessorHost()); - HStoreFile storeFile = new HStoreFile(info, getColumnFamilyDescriptor().getBloomFilterType(), - getCacheConfig()); - storeFile.initReader(); - return storeFile; + storeEngine.refreshStoreFiles(newFiles); + postRefreshStoreFiles(); } /** @@ -721,7 +530,7 @@ public void stopReplayingFromWAL(){ * Adds a value to the memstore */ public void add(final Cell cell, MemStoreSizing memstoreSizing) { - lock.readLock().lock(); + storeEngine.readLock(); try { if (this.currentParallelPutCount.getAndIncrement() > this.parallelPutCountPrintThreshold) { LOG.trace("tableName={}, encodedName={}, columnFamilyName={} is too busy!", @@ -729,7 +538,7 @@ public void add(final Cell cell, MemStoreSizing memstoreSizing) { } this.memstore.add(cell, memstoreSizing); } finally { - lock.readLock().unlock(); + storeEngine.readUnlock(); currentParallelPutCount.decrementAndGet(); } } @@ -738,7 +547,7 @@ public void add(final Cell cell, MemStoreSizing memstoreSizing) { * Adds the specified value to the memstore */ public void add(final Iterable cells, MemStoreSizing memstoreSizing) { - lock.readLock().lock(); + storeEngine.readLock(); try { if (this.currentParallelPutCount.getAndIncrement() > this.parallelPutCountPrintThreshold) { LOG.trace("tableName={}, encodedName={}, columnFamilyName={} is too busy!", @@ -746,7 +555,7 @@ public void add(final Iterable cells, MemStoreSizing memstoreSizing) { } memstore.add(cells, memstoreSizing); } finally { - lock.readLock().unlock(); + storeEngine.readUnlock(); currentParallelPutCount.decrementAndGet(); } } @@ -869,17 +678,16 @@ public Path bulkLoadHFile(byte[] family, String srcPathStr, Path dstPath) throws LOG.info("Loaded HFile " + srcPath + " into " + this + " as " + dstPath + " - updating store file list."); - HStoreFile sf = createStoreFileAndReader(dstPath); + HStoreFile sf = storeEngine.createStoreFileAndReader(dstPath); bulkLoadHFile(sf); - LOG.info("Successfully loaded {} into {} (new location: {})", - srcPath, this, dstPath); + LOG.info("Successfully loaded {} into {} (new location: {})", srcPath, this, dstPath); return dstPath; } public void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException { - HStoreFile sf = createStoreFileAndReader(fileInfo); + HStoreFile sf = storeEngine.createStoreFileAndReader(fileInfo); bulkLoadHFile(sf); } @@ -887,28 +695,75 @@ private void bulkLoadHFile(HStoreFile sf) throws IOException { StoreFileReader r = sf.getReader(); this.storeSize.addAndGet(r.length()); this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes()); - - // Append the new storefile into the list - this.lock.writeLock().lock(); - try { - this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf)); - } finally { - // We need the lock, as long as we are updating the storeFiles - // or changing the memstore. Let us release it before calling - // notifyChangeReadersObservers. See HBASE-4485 for a possible - // deadlock scenario that could have happened if continue to hold - // the lock. - this.lock.writeLock().unlock(); - } + storeEngine.addStoreFiles(Lists.newArrayList(sf), () -> { + }); LOG.info("Loaded HFile " + sf.getFileInfo() + " into " + this); if (LOG.isTraceEnabled()) { - String traceMessage = "BULK LOAD time,size,store size,store files [" - + EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize - + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]"; + String traceMessage = "BULK LOAD time,size,store size,store files [" + + EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize + "," + + storeEngine.getStoreFileManager().getStorefileCount() + "]"; LOG.trace(traceMessage); } } + private ImmutableCollection closeWithoutLock() throws IOException { + // Clear so metrics doesn't find them. + ImmutableCollection result = storeEngine.getStoreFileManager().clearFiles(); + Collection compactedfiles = storeEngine.getStoreFileManager().clearCompactedFiles(); + // clear the compacted files + if (CollectionUtils.isNotEmpty(compactedfiles)) { + removeCompactedfiles(compactedfiles, + getCacheConfig() != null ? getCacheConfig().shouldEvictOnClose() : true); + } + if (!result.isEmpty()) { + // initialize the thread pool for closing store files in parallel. + ThreadPoolExecutor storeFileCloserThreadPool = + this.region.getStoreFileOpenAndCloseThreadPool("StoreFileCloser-" + + this.region.getRegionInfo().getEncodedName() + "-" + this.getColumnFamilyName()); + + // close each store file in parallel + CompletionService completionService = + new ExecutorCompletionService<>(storeFileCloserThreadPool); + for (HStoreFile f : result) { + completionService.submit(new Callable() { + @Override + public Void call() throws IOException { + boolean evictOnClose = + getCacheConfig() != null ? getCacheConfig().shouldEvictOnClose() : true; + f.closeStoreFile(evictOnClose); + return null; + } + }); + } + + IOException ioe = null; + try { + for (int i = 0; i < result.size(); i++) { + try { + Future future = completionService.take(); + future.get(); + } catch (InterruptedException e) { + if (ioe == null) { + ioe = new InterruptedIOException(); + ioe.initCause(e); + } + } catch (ExecutionException e) { + if (ioe == null) { + ioe = new IOException(e.getCause()); + } + } + } + } finally { + storeFileCloserThreadPool.shutdownNow(); + } + if (ioe != null) { + throw ioe; + } + } + LOG.trace("Closed {}", this); + return result; + } + /** * Close all the readers We don't need to worry about subsequent requests because the Region holds * a write lock that will prevent any more reads or writes. @@ -916,67 +771,18 @@ private void bulkLoadHFile(HStoreFile sf) throws IOException { * @throws IOException on failure */ public ImmutableCollection close() throws IOException { + // findbugs can not recognize storeEngine.writeLock is just a lock operation so it will report + // UL_UNRELEASED_LOCK_EXCEPTION_PATH, so here we have to use two try finally... + // Change later if findbugs becomes smarter in the future. this.archiveLock.lock(); - this.lock.writeLock().lock(); try { - // Clear so metrics doesn't find them. - ImmutableCollection result = storeEngine.getStoreFileManager().clearFiles(); - Collection compactedfiles = - storeEngine.getStoreFileManager().clearCompactedFiles(); - // clear the compacted files - if (CollectionUtils.isNotEmpty(compactedfiles)) { - removeCompactedfiles(compactedfiles, getCacheConfig() != null ? - getCacheConfig().shouldEvictOnClose() : true); - } - if (!result.isEmpty()) { - // initialize the thread pool for closing store files in parallel. - ThreadPoolExecutor storeFileCloserThreadPool = this.region - .getStoreFileOpenAndCloseThreadPool("StoreFileCloser-" - + this.region.getRegionInfo().getEncodedName() + "-" + this.getColumnFamilyName()); - - // close each store file in parallel - CompletionService completionService = - new ExecutorCompletionService<>(storeFileCloserThreadPool); - for (HStoreFile f : result) { - completionService.submit(new Callable() { - @Override - public Void call() throws IOException { - boolean evictOnClose = - getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true; - f.closeStoreFile(evictOnClose); - return null; - } - }); - } - - IOException ioe = null; - try { - for (int i = 0; i < result.size(); i++) { - try { - Future future = completionService.take(); - future.get(); - } catch (InterruptedException e) { - if (ioe == null) { - ioe = new InterruptedIOException(); - ioe.initCause(e); - } - } catch (ExecutionException e) { - if (ioe == null) { - ioe = new IOException(e.getCause()); - } - } - } - } finally { - storeFileCloserThreadPool.shutdownNow(); - } - if (ioe != null) { - throw ioe; - } + this.storeEngine.writeLock(); + try { + return closeWithoutLock(); + } finally { + this.storeEngine.writeUnlock(); } - LOG.trace("Closed {}", this); - return result; } finally { - this.lock.writeLock().unlock(); this.archiveLock.unlock(); } } @@ -1006,7 +812,7 @@ protected List flushCache(final long logCacheFlushId, MemStoreSnapshot sna try { for (Path pathName : pathNames) { lastPathName = pathName; - validateStoreFile(pathName); + storeEngine.validateStoreFile(pathName); } return pathNames; } catch (Exception e) { @@ -1052,204 +858,37 @@ public HStoreFile tryCommitRecoveredHFile(Path path) throws IOException { } Path dstPath = getRegionFileSystem().commitStoreFile(getColumnFamilyName(), path); - HStoreFile sf = createStoreFileAndReader(dstPath); + HStoreFile sf = storeEngine.createStoreFileAndReader(dstPath); StoreFileReader r = sf.getReader(); this.storeSize.addAndGet(r.length()); this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes()); - this.lock.writeLock().lock(); - try { - this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf)); - } finally { - this.lock.writeLock().unlock(); - } + storeEngine.addStoreFiles(Lists.newArrayList(sf), () -> { + }); LOG.info("Loaded recovered hfile to {}, entries={}, sequenceid={}, filesize={}", sf, r.getEntries(), r.getSequenceID(), TraditionalBinaryPrefix.long2String(r.length(), "B", 1)); return sf; } - /** - * Commit the given {@code files}. - *

- * We will move the file into data directory, and open it. - * @param files the files want to commit - * @param validate whether to validate the store files - * @return the committed store files - */ - private List commitStoreFiles(List files, boolean validate) throws IOException { - List committedFiles = new ArrayList<>(files.size()); - HRegionFileSystem hfs = getRegionFileSystem(); - String familyName = getColumnFamilyName(); - for (Path file : files) { - try { - if (validate) { - validateStoreFile(file); - } - Path committedPath = hfs.commitStoreFile(familyName, file); - HStoreFile sf = createStoreFileAndReader(committedPath); - committedFiles.add(sf); - } catch (IOException e) { - LOG.error("Failed to commit store file {}", file, e); - // Try to delete the files we have committed before. - // It is OK to fail when deleting as leaving the file there does not cause any data - // corruption problem. It just introduces some duplicated data which may impact read - // performance a little when reading before compaction. - for (HStoreFile sf : committedFiles) { - Path pathToDelete = sf.getPath(); - try { - sf.deleteStoreFile(); - } catch (IOException deleteEx) { - LOG.warn(HBaseMarkers.FATAL, "Failed to delete committed store file {}", pathToDelete, - deleteEx); - } - } - throw new IOException("Failed to commit the flush", e); - } - } - return committedFiles; - } - - public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, - boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag, - boolean shouldDropBehind) throws IOException { - return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint, - includesTag, shouldDropBehind, -1, HConstants.EMPTY_STRING); - } - - /** - * @param compression Compression algorithm to use - * @param isCompaction whether we are creating a new file in a compaction - * @param includeMVCCReadpoint - whether to include MVCC or not - * @param includesTag - includesTag or not - * @return Writer for a new StoreFile in the tmp dir. - */ - // TODO : allow the Writer factory to create Writers of ShipperListener type only in case of - // compaction - public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, - boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag, - boolean shouldDropBehind, long totalCompactedFilesSize, String fileStoragePolicy) - throws IOException { - // creating new cache config for each new writer - final CacheConfig cacheConf = getCacheConfig(); - final CacheConfig writerCacheConf = new CacheConfig(cacheConf); - if (isCompaction) { - // Don't cache data on write on compactions, unless specifically configured to do so - // Cache only when total file size remains lower than configured threshold - final boolean cacheCompactedBlocksOnWrite = - getCacheConfig().shouldCacheCompactedBlocksOnWrite(); - // if data blocks are to be cached on write - // during compaction, we should forcefully - // cache index and bloom blocks as well - if (cacheCompactedBlocksOnWrite && totalCompactedFilesSize <= cacheConf - .getCacheCompactedBlocksOnWriteThreshold()) { - writerCacheConf.enableCacheOnWrite(); - if (!cacheOnWriteLogged) { - LOG.info("For {} , cacheCompactedBlocksOnWrite is true, hence enabled " + - "cacheOnWrite for Data blocks, Index blocks and Bloom filter blocks", this); - cacheOnWriteLogged = true; - } - } else { - writerCacheConf.setCacheDataOnWrite(false); - if (totalCompactedFilesSize > cacheConf.getCacheCompactedBlocksOnWriteThreshold()) { - // checking condition once again for logging - LOG.debug( - "For {}, setting cacheCompactedBlocksOnWrite as false as total size of compacted " - + "files - {}, is greater than cacheCompactedBlocksOnWriteThreshold - {}", - this, totalCompactedFilesSize, - cacheConf.getCacheCompactedBlocksOnWriteThreshold()); - } - } - } else { - final boolean shouldCacheDataOnWrite = cacheConf.shouldCacheDataOnWrite(); - if (shouldCacheDataOnWrite) { - writerCacheConf.enableCacheOnWrite(); - if (!cacheOnWriteLogged) { - LOG.info("For {} , cacheDataOnWrite is true, hence enabled cacheOnWrite for " + - "Index blocks and Bloom filter blocks", this); - cacheOnWriteLogged = true; - } - } - } - Encryption.Context encryptionContext = storeContext.getEncryptionContext(); - HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag, - encryptionContext); - Path familyTempDir = new Path(getRegionFileSystem().getTempDir(), getColumnFamilyName()); - StoreFileWriter.Builder builder = - new StoreFileWriter.Builder(conf, writerCacheConf, getFileSystem()) - .withOutputDir(familyTempDir) - .withBloomType(storeContext.getBloomFilterType()) - .withMaxKeyCount(maxKeyCount) - .withFavoredNodes(storeContext.getFavoredNodes()) - .withFileContext(hFileContext) - .withShouldDropCacheBehind(shouldDropBehind) - .withCompactedFilesSupplier(storeContext.getCompactedFilesSupplier()) - .withFileStoragePolicy(fileStoragePolicy); - return builder.build(); - } - - private HFileContext createFileContext(Compression.Algorithm compression, - boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context encryptionContext) { - if (compression == null) { - compression = HFile.DEFAULT_COMPRESSION_ALGORITHM; - } - ColumnFamilyDescriptor family = getColumnFamilyDescriptor(); - HFileContext hFileContext = new HFileContextBuilder() - .withIncludesMvcc(includeMVCCReadpoint) - .withIncludesTags(includesTag) - .withCompression(compression) - .withCompressTags(family.isCompressTags()) - .withChecksumType(StoreUtils.getChecksumType(conf)) - .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf)) - .withBlockSize(family.getBlocksize()) - .withHBaseCheckSum(true) - .withDataBlockEncoding(family.getDataBlockEncoding()) - .withEncryptionContext(encryptionContext) - .withCreateTime(EnvironmentEdgeManager.currentTime()) - .withColumnFamily(getColumnFamilyDescriptor().getName()) - .withTableName(getTableName().getName()) - .withCellComparator(getComparator()) - .build(); - return hFileContext; - } - private long getTotalSize(Collection sfs) { return sfs.stream().mapToLong(sf -> sf.getReader().length()).sum(); } - /** - * Change storeFiles adding into place the Reader produced by this new flush. - * @param sfs Store files - * @return Whether compaction is required. - */ - private boolean updateStorefiles(List sfs, long snapshotId) throws IOException { - this.lock.writeLock().lock(); - try { - this.storeEngine.getStoreFileManager().insertNewFiles(sfs); - /** - * NOTE:we should keep clearSnapshot method inside the write lock because clearSnapshot may - * close {@link DefaultMemStore#snapshot}, which may be used by - * {@link DefaultMemStore#getScanners}. - */ - if (snapshotId > 0) { - this.memstore.clearSnapshot(snapshotId); - } - } finally { - // We need the lock, as long as we are updating the storeFiles - // or changing the memstore. Let us release it before calling - // notifyChangeReadersObservers. See HBASE-4485 for a possible - // deadlock scenario that could have happened if continue to hold - // the lock. - this.lock.writeLock().unlock(); - } - + private boolean completeFlush(List sfs, long snapshotId) throws IOException { + // NOTE:we should keep clearSnapshot method inside the write lock because clearSnapshot may + // close {@link DefaultMemStore#snapshot}, which may be used by + // {@link DefaultMemStore#getScanners}. + storeEngine.addStoreFiles(sfs, + snapshotId > 0 ? () -> this.memstore.clearSnapshot(snapshotId) : () -> { + }); // notify to be called here - only in case of flushes notifyChangedReadersObservers(sfs); if (LOG.isTraceEnabled()) { long totalSize = getTotalSize(sfs); - String traceMessage = "FLUSH time,count,size,store size,store files [" - + EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize - + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]"; + String traceMessage = "FLUSH time,count,size,store size,store files [" + + EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize + "," + + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]"; LOG.trace(traceMessage); } return needsCompaction(); @@ -1261,11 +900,11 @@ private boolean updateStorefiles(List sfs, long snapshotId) throws I private void notifyChangedReadersObservers(List sfs) throws IOException { for (ChangedReadersObserver o : this.changedReaderObservers) { List memStoreScanners; - this.lock.readLock().lock(); + this.storeEngine.readLock(); try { memStoreScanners = this.memstore.getScanners(o.getReadPoint()); } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } o.updateReaders(sfs, memStoreScanners); } @@ -1307,13 +946,13 @@ public List getScanners(boolean cacheBlocks, boolean usePread, byte[] stopRow, boolean includeStopRow, long readPt) throws IOException { Collection storeFilesToScan; List memStoreScanners; - this.lock.readLock().lock(); + this.storeEngine.readLock(); try { storeFilesToScan = this.storeEngine.getStoreFileManager().getFilesForScan(startRow, includeStartRow, stopRow, includeStopRow); memStoreScanners = this.memstore.getScanners(readPt); } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } try { @@ -1390,11 +1029,11 @@ public List getScanners(List files, boolean cacheBl boolean includeMemstoreScanner) throws IOException { List memStoreScanners = null; if (includeMemstoreScanner) { - this.lock.readLock().lock(); + this.storeEngine.readLock(); try { memStoreScanners = this.memstore.getScanners(readPt); } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } } try { @@ -1510,14 +1149,13 @@ protected List doCompaction(CompactionRequestImpl cr, List newFiles) throws IOException { // Do the steps necessary to complete the compaction. setStoragePolicyFromFileName(newFiles); - List sfs = commitStoreFiles(newFiles, true); + List sfs = storeEngine.commitStoreFiles(newFiles, true); if (this.getCoprocessorHost() != null) { for (HStoreFile sf : sfs) { getCoprocessorHost().postCompact(this, sf, cr.getTracker(), cr, user); } } - writeCompactionWalRecord(filesToCompact, sfs); - replaceStoreFiles(filesToCompact, sfs); + replaceStoreFiles(filesToCompact, sfs, true); if (cr.isMajor()) { majorCompactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs()); majorCompactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize); @@ -1581,25 +1219,24 @@ private void writeCompactionWalRecord(Collection filesCompacted, this.region.getRegionInfo(), compactionDescriptor, this.region.getMVCC()); } - void replaceStoreFiles(Collection compactedFiles, Collection result) - throws IOException { - this.lock.writeLock().lock(); - try { - this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result); - synchronized (filesCompacting) { - filesCompacting.removeAll(compactedFiles); - } - - // These may be null when the RS is shutting down. The space quota Chores will fix the Region - // sizes later so it's not super-critical if we miss these. - RegionServerServices rsServices = region.getRegionServerServices(); - if (rsServices != null && rsServices.getRegionServerSpaceQuotaManager() != null) { - updateSpaceQuotaAfterFileReplacement( - rsServices.getRegionServerSpaceQuotaManager().getRegionSizeStore(), getRegionInfo(), - compactedFiles, result); - } - } finally { - this.lock.writeLock().unlock(); + @RestrictedApi(explanation = "Should only be called in TestHStore", link = "", + allowedOnPath = ".*/(HStore|TestHStore).java") + void replaceStoreFiles(Collection compactedFiles, Collection result, + boolean writeCompactionMarker) throws IOException { + storeEngine.replaceStoreFiles(compactedFiles, result); + if (writeCompactionMarker) { + writeCompactionWalRecord(compactedFiles, result); + } + synchronized (filesCompacting) { + filesCompacting.removeAll(compactedFiles); + } + // These may be null when the RS is shutting down. The space quota Chores will fix the Region + // sizes later so it's not super-critical if we miss these. + RegionServerServices rsServices = region.getRegionServerServices(); + if (rsServices != null && rsServices.getRegionServerSpaceQuotaManager() != null) { + updateSpaceQuotaAfterFileReplacement( + rsServices.getRegionServerSpaceQuotaManager().getRegionSizeStore(), getRegionInfo(), + compactedFiles, result); } } @@ -1722,7 +1359,7 @@ public void replayCompactionMarker(CompactionDescriptor compaction, boolean pick for (String compactionOutput : compactionOutputs) { StoreFileInfo storeFileInfo = getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), compactionOutput); - HStoreFile storeFile = createStoreFileAndReader(storeFileInfo); + HStoreFile storeFile = storeEngine.createStoreFileAndReader(storeFileInfo); outputStoreFiles.add(storeFile); } } @@ -1730,7 +1367,7 @@ public void replayCompactionMarker(CompactionDescriptor compaction, boolean pick if (!inputStoreFiles.isEmpty() || !outputStoreFiles.isEmpty()) { LOG.info("Replaying compaction marker, replacing input files: " + inputStoreFiles + " with output files : " + outputStoreFiles); - this.replaceStoreFiles(inputStoreFiles, outputStoreFiles); + this.replaceStoreFiles(inputStoreFiles, outputStoreFiles, false); this.refreshStoreSizeAndTotalBytes(); } } @@ -1739,14 +1376,14 @@ public void replayCompactionMarker(CompactionDescriptor compaction, boolean pick public boolean hasReferences() { // Grab the read lock here, because we need to ensure that: only when the atomic // replaceStoreFiles(..) finished, we can get all the complete store file list. - this.lock.readLock().lock(); + this.storeEngine.readLock(); try { // Merge the current store files with compacted files here due to HBASE-20940. Collection allStoreFiles = new ArrayList<>(getStorefiles()); allStoreFiles.addAll(getCompactedFiles()); return StoreUtils.hasReferences(allStoreFiles); } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } } @@ -1786,7 +1423,7 @@ public Optional requestCompaction(int priority, final CompactionContext compaction = storeEngine.createCompaction(); CompactionRequestImpl request = null; - this.lock.readLock().lock(); + this.storeEngine.readLock(); try { synchronized (filesCompacting) { // First, see if coprocessor would want to override selection. @@ -1859,7 +1496,7 @@ public Optional requestCompaction(int priority, request.setTracker(tracker); } } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } if (LOG.isDebugEnabled()) { @@ -1892,7 +1529,7 @@ private void removeUnneededFiles() throws IOException { this, getColumnFamilyDescriptor().getMinVersions()); return; } - this.lock.readLock().lock(); + this.storeEngine.readLock(); Collection delSfs = null; try { synchronized (filesCompacting) { @@ -1904,7 +1541,7 @@ private void removeUnneededFiles() throws IOException { } } } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } if (CollectionUtils.isEmpty(delSfs)) { @@ -1912,8 +1549,7 @@ private void removeUnneededFiles() throws IOException { } Collection newFiles = Collections.emptyList(); // No new files. - writeCompactionWalRecord(delSfs, newFiles); - replaceStoreFiles(delSfs, newFiles); + replaceStoreFiles(delSfs, newFiles, true); refreshStoreSizeAndTotalBytes(); LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in " + this + "; total size is " @@ -1935,25 +1571,6 @@ protected void finishCompactionRequest(CompactionRequestImpl cr) { } } - /** - * Validates a store file by opening and closing it. In HFileV2 this should not be an expensive - * operation. - * @param path the path to the store file - */ - private void validateStoreFile(Path path) throws IOException { - HStoreFile storeFile = null; - try { - storeFile = createStoreFileAndReader(path); - } catch (IOException e) { - LOG.error("Failed to open store file : {}, keeping it in tmp location", path, e); - throw e; - } finally { - if (storeFile != null) { - storeFile.closeStoreFile(false); - } - } - } - /** * Update counts. */ @@ -1999,7 +1616,7 @@ public boolean canSplit() { * Determines if Store should be split. */ public Optional getSplitPoint() { - this.lock.readLock().lock(); + this.storeEngine.readLock(); try { // Should already be enforced by the split policy! assert !this.getRegionInfo().isMetaRegion(); @@ -2012,7 +1629,7 @@ public Optional getSplitPoint() { } catch(IOException e) { LOG.warn("Failed getting store size for {}", this, e); } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } return Optional.empty(); } @@ -2045,7 +1662,7 @@ public void triggerMajorCompaction() { */ public KeyValueScanner getScanner(Scan scan, final NavigableSet targetCols, long readPt) throws IOException { - lock.readLock().lock(); + storeEngine.readLock(); try { ScanInfo scanInfo; if (this.getCoprocessorHost() != null) { @@ -2055,7 +1672,7 @@ public KeyValueScanner getScanner(Scan scan, final NavigableSet targetCo } return createScanner(scan, scanInfo, targetCols, readPt); } finally { - lock.readLock().unlock(); + storeEngine.readUnlock(); } } @@ -2085,7 +1702,7 @@ public List recreateScanners(List currentFileS boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner) throws IOException { - this.lock.readLock().lock(); + this.storeEngine.readLock(); try { Map name2File = new HashMap<>(getStorefilesCount() + getCompactedFilesCount()); @@ -2110,7 +1727,7 @@ public List recreateScanners(List currentFileS return getScanners(filesToReopen, cacheBlocks, false, false, matcher, startRow, includeStartRow, stopRow, includeStopRow, readPt, false); } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } } @@ -2176,41 +1793,20 @@ public long getStoreSizeUncompressed() { @Override public long getStorefilesSize() { // Include all StoreFiles - return getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(), sf -> true); + return StoreUtils.getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(), + sf -> true); } @Override public long getHFilesSize() { // Include only StoreFiles which are HFiles - return getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(), + return StoreUtils.getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(), HStoreFile::isHFile); } - private long getTotalUncompressedBytes(List files) { - return files.stream() - .mapToLong(file -> getStorefileFieldSize(file, StoreFileReader::getTotalUncompressedBytes)) - .sum(); - } - - private long getStorefilesSize(Collection files, Predicate predicate) { - return files.stream().filter(predicate) - .mapToLong(file -> getStorefileFieldSize(file, StoreFileReader::length)).sum(); - } - - private long getStorefileFieldSize(HStoreFile file, ToLongFunction f) { - if (file == null) { - return 0L; - } - StoreFileReader reader = file.getReader(); - if (reader == null) { - return 0L; - } - return f.applyAsLong(reader); - } - private long getStorefilesFieldSize(ToLongFunction f) { return this.storeEngine.getStoreFileManager().getStorefiles().stream() - .mapToLong(file -> getStorefileFieldSize(file, f)).sum(); + .mapToLong(file -> StoreUtils.getStorefileFieldSize(file, f)).sum(); } @Override @@ -2281,11 +1877,11 @@ public long getSmallestReadPoint() { */ public void upsert(Iterable cells, long readpoint, MemStoreSizing memstoreSizing) throws IOException { - this.lock.readLock().lock(); + this.storeEngine.readLock(); try { this.memstore.upsert(cells, readpoint, memstoreSizing); } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } } @@ -2338,7 +1934,7 @@ public boolean commit(MonitoredTask status) throws IOException { return false; } status.setStatus("Flushing " + this + ": reopening flushed file"); - List storeFiles = commitStoreFiles(tempFiles, false); + List storeFiles = storeEngine.commitStoreFiles(tempFiles, false); for (HStoreFile sf : storeFiles) { StoreFileReader r = sf.getReader(); if (LOG.isInfoEnabled()) { @@ -2361,7 +1957,7 @@ public boolean commit(MonitoredTask status) throws IOException { } } // Add new file to store files. Clear snapshot too while we have the Store write lock. - return updateStorefiles(storeFiles, snapshot.getId()); + return completeFlush(storeFiles, snapshot.getId()); } @Override @@ -2389,7 +1985,7 @@ public void replayFlush(List fileNames, boolean dropMemstoreSnapshot) // open the file as a store file (hfile link, etc) StoreFileInfo storeFileInfo = getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), file); - HStoreFile storeFile = createStoreFileAndReader(storeFileInfo); + HStoreFile storeFile = storeEngine.createStoreFileAndReader(storeFileInfo); storeFiles.add(storeFile); HStore.this.storeSize.addAndGet(storeFile.getReader().length()); HStore.this.totalUncompressedBytes @@ -2406,7 +2002,7 @@ public void replayFlush(List fileNames, boolean dropMemstoreSnapshot) snapshotId = snapshot.getId(); snapshot.close(); } - HStore.this.updateStorefiles(storeFiles, snapshotId); + HStore.this.completeFlush(storeFiles, snapshotId); } /** @@ -2419,7 +2015,7 @@ public void abort() throws IOException { //won't be closed. If we are using MSLAB, the chunk referenced by those scanners //can't be released, thus memory leak snapshot.close(); - HStore.this.updateStorefiles(Collections.emptyList(), snapshot.getId()); + HStore.this.completeFlush(Collections.emptyList(), snapshot.getId()); } } } @@ -2582,7 +2178,7 @@ public synchronized void closeAndArchiveCompactedFiles() throws IOException { // ensure other threads do not attempt to archive the same files on close() archiveLock.lock(); try { - lock.readLock().lock(); + storeEngine.readLock(); Collection copyCompactedfiles = null; try { Collection compactedfiles = @@ -2594,7 +2190,7 @@ public synchronized void closeAndArchiveCompactedFiles() throws IOException { LOG.trace("No compacted files to archive"); } } finally { - lock.readLock().unlock(); + storeEngine.readUnlock(); } if (CollectionUtils.isNotEmpty(copyCompactedfiles)) { removeCompactedfiles(copyCompactedfiles, true); @@ -2729,12 +2325,7 @@ public boolean isSloppyMemStore() { private void clearCompactedfiles(List filesToRemove) throws IOException { LOG.trace("Clearing the compacted file {} from this store", filesToRemove); - try { - lock.writeLock().lock(); - this.getStoreEngine().getStoreFileManager().removeCompactedFiles(filesToRemove); - } finally { - lock.writeLock().unlock(); - } + storeEngine.removeCompactedFiles(filesToRemove); } void reportArchivedFilesForQuota(List archivedFiles, List fileSizes) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java index 26233505db73..2a9f96859361 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -108,6 +109,14 @@ public RegionCoprocessorHost getCoprocessorHost() { return coprocessorHost; } + public RegionInfo getRegionInfo() { + return regionFileSystem.getRegionInfo(); + } + + public boolean isPrimaryReplicaStore() { + return getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID; + } + public static Builder getBuilder() { return new Builder(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java index 60b3c3d0d20f..4033c335d52c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java @@ -19,38 +19,131 @@ package org.apache.hadoop.hbase.regionserver; +import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; - +import java.util.Set; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.Compactor; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; +import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; /** - * StoreEngine is a factory that can create the objects necessary for HStore to operate. - * Since not all compaction policies, compactors and store file managers are compatible, - * they are tied together and replaced together via StoreEngine-s. + * StoreEngine is a factory that can create the objects necessary for HStore to operate. Since not + * all compaction policies, compactors and store file managers are compatible, they are tied + * together and replaced together via StoreEngine-s. + *

+ * We expose read write lock methods to upper layer for store operations:
+ *

    + *
  • Locked in shared mode when the list of component stores is looked at: + *
      + *
    • all reads/writes to table data
    • + *
    • checking for split
    • + *
    + *
  • + *
  • Locked in exclusive mode when the list of component stores is modified: + *
      + *
    • closing
    • + *
    • completing a compaction
    • + *
    + *
  • + *
+ *

+ * It is a bit confusing that we have a StoreFileManager(SFM) and then a StoreFileTracker(SFT). As + * its name says, SFT is used to track the store files list. The reason why we have a SFT beside SFM + * is that, when introducing stripe compaction, we introduced the StoreEngine and also the SFM, but + * actually, the SFM here is not a general 'Manager', it is only designed to manage the in memory + * 'stripes', so we can select different store files when scanning or compacting. The 'tracking' of + * store files is actually done in {@link org.apache.hadoop.hbase.regionserver.HRegionFileSystem} + * and {@link HStore} before we have SFT. And since SFM is designed to only holds in memory states, + * we will hold write lock when updating it, the lock is also used to protect the normal read/write + * requests. This means we'd better not add IO operations to SFM. And also, no matter what the in + * memory state is, stripe or not, it does not effect how we track the store files. So consider all + * these facts, here we introduce a separated SFT to track the store files. + *

+ * Here, since we always need to update SFM and SFT almost at the same time, we introduce methods in + * StoreEngine directly to update them both, so upper layer just need to update StoreEngine once, to + * reduce the possible misuse. */ @InterfaceAudience.Private -public abstract class StoreEngine { +public abstract class StoreEngine { + + private static final Logger LOG = LoggerFactory.getLogger(StoreEngine.class); + protected SF storeFlusher; protected CP compactionPolicy; protected C compactor; protected SFM storeFileManager; + private Configuration conf; + private StoreContext ctx; + private RegionCoprocessorHost coprocessorHost; + private Function openStoreFileThreadPoolCreator; + private StoreFileTracker storeFileTracker; + + private final ReadWriteLock storeLock = new ReentrantReadWriteLock(); /** - * The name of the configuration parameter that specifies the class of - * a store engine that is used to manage and compact HBase store files. + * The name of the configuration parameter that specifies the class of a store engine that is used + * to manage and compact HBase store files. */ public static final String STORE_ENGINE_CLASS_KEY = "hbase.hstore.engine.class"; - private static final Class> - DEFAULT_STORE_ENGINE_CLASS = DefaultStoreEngine.class; + private static final Class> DEFAULT_STORE_ENGINE_CLASS = + DefaultStoreEngine.class; + + /** + * Acquire read lock of this store. + */ + public void readLock() { + storeLock.readLock().lock(); + } + + /** + * Release read lock of this store. + */ + public void readUnlock() { + storeLock.readLock().unlock(); + } + + /** + * Acquire write lock of this store. + */ + public void writeLock() { + storeLock.writeLock().lock(); + } + + /** + * Release write lock of this store. + */ + public void writeUnlock() { + storeLock.writeLock().unlock(); + } /** * @return Compaction policy to use. @@ -80,6 +173,11 @@ public StoreFlusher getStoreFlusher() { return this.storeFlusher; } + private StoreFileTracker createStoreFileTracker(HStore store) { + return StoreFileTrackerFactory.create(store.conf, store.getRegionInfo().getTable(), + store.isPrimaryReplicaStore(), store.getStoreContext()); + } + /** * @param filesCompacting Files currently compacting * @return whether a compaction selection is possible @@ -87,8 +185,8 @@ public StoreFlusher getStoreFlusher() { public abstract boolean needsCompaction(List filesCompacting); /** - * Creates an instance of a compaction context specific to this engine. - * Doesn't actually select or start a compaction. See CompactionContext class comment. + * Creates an instance of a compaction context specific to this engine. Doesn't actually select or + * start a compaction. See CompactionContext class comment. * @return New CompactionContext object. */ public abstract CompactionContext createCompaction() throws IOException; @@ -96,36 +194,347 @@ public StoreFlusher getStoreFlusher() { /** * Create the StoreEngine's components. */ - protected abstract void createComponents( - Configuration conf, HStore store, CellComparator cellComparator) throws IOException; + protected abstract void createComponents(Configuration conf, HStore store, + CellComparator cellComparator) throws IOException; - private void createComponentsOnce( - Configuration conf, HStore store, CellComparator cellComparator) throws IOException { - assert compactor == null && compactionPolicy == null - && storeFileManager == null && storeFlusher == null; + protected final void createComponentsOnce(Configuration conf, HStore store, + CellComparator cellComparator) throws IOException { + assert compactor == null && compactionPolicy == null && storeFileManager == null && + storeFlusher == null && storeFileTracker == null; createComponents(conf, store, cellComparator); - assert compactor != null && compactionPolicy != null - && storeFileManager != null && storeFlusher != null; + this.conf = conf; + this.ctx = store.getStoreContext(); + this.coprocessorHost = store.getHRegion().getCoprocessorHost(); + this.openStoreFileThreadPoolCreator = store.getHRegion()::getStoreFileOpenAndCloseThreadPool; + this.storeFileTracker = createStoreFileTracker(store); + assert compactor != null && compactionPolicy != null && storeFileManager != null && + storeFlusher != null && storeFileTracker != null; + } + + /** + * Create a writer for writing new store files. + * @return Writer for a new StoreFile + */ + public StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException { + return storeFileTracker.createWriter(params); + } + + public HStoreFile createStoreFileAndReader(Path p) throws IOException { + StoreFileInfo info = new StoreFileInfo(conf, ctx.getRegionFileSystem().getFileSystem(), p, + ctx.isPrimaryReplicaStore()); + return createStoreFileAndReader(info); + } + + public HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException { + info.setRegionCoprocessorHost(coprocessorHost); + HStoreFile storeFile = + new HStoreFile(info, ctx.getFamily().getBloomFilterType(), ctx.getCacheConf()); + storeFile.initReader(); + return storeFile; + } + + /** + * Validates a store file by opening and closing it. In HFileV2 this should not be an expensive + * operation. + * @param path the path to the store file + */ + public void validateStoreFile(Path path) throws IOException { + HStoreFile storeFile = null; + try { + storeFile = createStoreFileAndReader(path); + } catch (IOException e) { + LOG.error("Failed to open store file : {}, keeping it in tmp location", path, e); + throw e; + } finally { + if (storeFile != null) { + storeFile.closeStoreFile(false); + } + } + } + + private List openStoreFiles(Collection files, boolean warmup) + throws IOException { + if (CollectionUtils.isEmpty(files)) { + return Collections.emptyList(); + } + // initialize the thread pool for opening store files in parallel.. + ExecutorService storeFileOpenerThreadPool = + openStoreFileThreadPoolCreator.apply("StoreFileOpener-" + + ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString()); + CompletionService completionService = + new ExecutorCompletionService<>(storeFileOpenerThreadPool); + + int totalValidStoreFile = 0; + for (StoreFileInfo storeFileInfo : files) { + // The StoreFileInfo will carry store configuration down to HFile, we need to set it to + // our store's CompoundConfiguration here. + storeFileInfo.setConf(conf); + // open each store file in parallel + completionService.submit(() -> createStoreFileAndReader(storeFileInfo)); + totalValidStoreFile++; + } + + Set compactedStoreFiles = new HashSet<>(); + ArrayList results = new ArrayList<>(files.size()); + IOException ioe = null; + try { + for (int i = 0; i < totalValidStoreFile; i++) { + try { + HStoreFile storeFile = completionService.take().get(); + if (storeFile != null) { + LOG.debug("loaded {}", storeFile); + results.add(storeFile); + compactedStoreFiles.addAll(storeFile.getCompactedStoreFiles()); + } + } catch (InterruptedException e) { + if (ioe == null) { + ioe = new InterruptedIOException(e.getMessage()); + } + } catch (ExecutionException e) { + if (ioe == null) { + ioe = new IOException(e.getCause()); + } + } + } + } finally { + storeFileOpenerThreadPool.shutdownNow(); + } + if (ioe != null) { + // close StoreFile readers + boolean evictOnClose = + ctx.getCacheConf() != null ? ctx.getCacheConf().shouldEvictOnClose() : true; + for (HStoreFile file : results) { + try { + if (file != null) { + file.closeStoreFile(evictOnClose); + } + } catch (IOException e) { + LOG.warn("Could not close store file {}", file, e); + } + } + throw ioe; + } + + // Should not archive the compacted store files when region warmup. See HBASE-22163. + if (!warmup) { + // Remove the compacted files from result + List filesToRemove = new ArrayList<>(compactedStoreFiles.size()); + for (HStoreFile storeFile : results) { + if (compactedStoreFiles.contains(storeFile.getPath().getName())) { + LOG.warn("Clearing the compacted storefile {} from {}", storeFile, this); + storeFile.getReader().close( + storeFile.getCacheConf() != null ? storeFile.getCacheConf().shouldEvictOnClose() : + true); + filesToRemove.add(storeFile); + } + } + results.removeAll(filesToRemove); + if (!filesToRemove.isEmpty() && ctx.isPrimaryReplicaStore()) { + LOG.debug("Moving the files {} to archive", filesToRemove); + ctx.getRegionFileSystem().removeStoreFiles(ctx.getFamily().getNameAsString(), + filesToRemove); + } + } + + return results; + } + + public void initialize(boolean warmup) throws IOException { + List fileInfos = storeFileTracker.load(); + List files = openStoreFiles(fileInfos, warmup); + storeFileManager.loadFiles(files); + } + + public void refreshStoreFiles() throws IOException { + List fileInfos = storeFileTracker.load(); + refreshStoreFilesInternal(fileInfos); + } + + public void refreshStoreFiles(Collection newFiles) throws IOException { + List storeFiles = new ArrayList<>(newFiles.size()); + for (String file : newFiles) { + storeFiles + .add(ctx.getRegionFileSystem().getStoreFileInfo(ctx.getFamily().getNameAsString(), file)); + } + refreshStoreFilesInternal(storeFiles); + } + + /** + * Checks the underlying store files, and opens the files that have not been opened, and removes + * the store file readers for store files no longer available. Mainly used by secondary region + * replicas to keep up to date with the primary region files. + */ + private void refreshStoreFilesInternal(Collection newFiles) throws IOException { + Collection currentFiles = storeFileManager.getStorefiles(); + Collection compactedFiles = storeFileManager.getCompactedfiles(); + if (currentFiles == null) { + currentFiles = Collections.emptySet(); + } + if (newFiles == null) { + newFiles = Collections.emptySet(); + } + if (compactedFiles == null) { + compactedFiles = Collections.emptySet(); + } + + HashMap currentFilesSet = new HashMap<>(currentFiles.size()); + for (HStoreFile sf : currentFiles) { + currentFilesSet.put(sf.getFileInfo(), sf); + } + HashMap compactedFilesSet = new HashMap<>(compactedFiles.size()); + for (HStoreFile sf : compactedFiles) { + compactedFilesSet.put(sf.getFileInfo(), sf); + } + + Set newFilesSet = new HashSet(newFiles); + // Exclude the files that have already been compacted + newFilesSet = Sets.difference(newFilesSet, compactedFilesSet.keySet()); + Set toBeAddedFiles = Sets.difference(newFilesSet, currentFilesSet.keySet()); + Set toBeRemovedFiles = Sets.difference(currentFilesSet.keySet(), newFilesSet); + + if (toBeAddedFiles.isEmpty() && toBeRemovedFiles.isEmpty()) { + return; + } + + LOG.info("Refreshing store files for " + this + " files to add: " + toBeAddedFiles + + " files to remove: " + toBeRemovedFiles); + + Set toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size()); + for (StoreFileInfo sfi : toBeRemovedFiles) { + toBeRemovedStoreFiles.add(currentFilesSet.get(sfi)); + } + + // try to open the files + List openedFiles = openStoreFiles(toBeAddedFiles, false); + + // propogate the file changes to the underlying store file manager + replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); // won't throw an exception + } + + /** + * Commit the given {@code files}. + *

+ * We will move the file into data directory, and open it. + * @param files the files want to commit + * @param validate whether to validate the store files + * @return the committed store files + */ + public List commitStoreFiles(List files, boolean validate) throws IOException { + List committedFiles = new ArrayList<>(files.size()); + HRegionFileSystem hfs = ctx.getRegionFileSystem(); + String familyName = ctx.getFamily().getNameAsString(); + Path storeDir = hfs.getStoreDir(familyName); + for (Path file : files) { + try { + if (validate) { + validateStoreFile(file); + } + Path committedPath; + // As we want to support writing to data directory directly, here we need to check whether + // the store file is already in the right place + if (file.getParent() != null && file.getParent().equals(storeDir)) { + // already in the right place, skip renmaing + committedPath = file; + } else { + // Write-out finished successfully, move into the right spot + committedPath = hfs.commitStoreFile(familyName, file); + } + HStoreFile sf = createStoreFileAndReader(committedPath); + committedFiles.add(sf); + } catch (IOException e) { + LOG.error("Failed to commit store file {}", file, e); + // Try to delete the files we have committed before. + // It is OK to fail when deleting as leaving the file there does not cause any data + // corruption problem. It just introduces some duplicated data which may impact read + // performance a little when reading before compaction. + for (HStoreFile sf : committedFiles) { + Path pathToDelete = sf.getPath(); + try { + sf.deleteStoreFile(); + } catch (IOException deleteEx) { + LOG.warn(HBaseMarkers.FATAL, "Failed to delete committed store file {}", pathToDelete, + deleteEx); + } + } + throw new IOException("Failed to commit the flush", e); + } + } + return committedFiles; + } + + @FunctionalInterface + public interface IOExceptionRunnable { + void run() throws IOException; + } + + /** + * Add the store files to store file manager, and also record it in the store file tracker. + *

+ * The {@code actionAfterAdding} will be executed after the insertion to store file manager, under + * the lock protection. Usually this is for clear the memstore snapshot. + */ + public void addStoreFiles(Collection storeFiles, + IOExceptionRunnable actionAfterAdding) throws IOException { + storeFileTracker.add(StoreUtils.toStoreFileInfo(storeFiles)); + writeLock(); + try { + storeFileManager.insertNewFiles(storeFiles); + actionAfterAdding.run(); + } finally { + // We need the lock, as long as we are updating the storeFiles + // or changing the memstore. Let us release it before calling + // notifyChangeReadersObservers. See HBASE-4485 for a possible + // deadlock scenario that could have happened if continue to hold + // the lock. + writeUnlock(); + } + } + + public void replaceStoreFiles(Collection compactedFiles, + Collection newFiles) throws IOException { + storeFileTracker.replace(StoreUtils.toStoreFileInfo(compactedFiles), + StoreUtils.toStoreFileInfo(newFiles)); + writeLock(); + try { + storeFileManager.addCompactionResults(compactedFiles, newFiles); + } finally { + writeUnlock(); + } + } + + public void removeCompactedFiles(Collection compactedFiles) { + writeLock(); + try { + storeFileManager.removeCompactedFiles(compactedFiles); + } finally { + writeUnlock(); + } } /** * Create the StoreEngine configured for the given Store. - * @param store The store. An unfortunate dependency needed due to it - * being passed to coprocessors via the compactor. + * @param store The store. An unfortunate dependency needed due to it being passed to coprocessors + * via the compactor. * @param conf Store configuration. * @param cellComparator CellComparator for storeFileManager. * @return StoreEngine to use. */ - public static StoreEngine create( - HStore store, Configuration conf, CellComparator cellComparator) throws IOException { + public static StoreEngine create(HStore store, Configuration conf, + CellComparator cellComparator) throws IOException { String className = conf.get(STORE_ENGINE_CLASS_KEY, DEFAULT_STORE_ENGINE_CLASS.getName()); try { - StoreEngine se = ReflectionUtils.instantiateWithCustomCtor( - className, new Class[] { }, new Object[] { }); + StoreEngine se = + ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {}, new Object[] {}); se.createComponentsOnce(conf, store, cellComparator); return se; } catch (Exception e) { throw new IOException("Unable to load configured store engine '" + className + "'", e); } } + + @RestrictedApi(explanation = "Should only be called in TestHStore", link = "", + allowedOnPath = ".*/TestHStore.java") + ReadWriteLock getLock() { + return storeLock; + } } 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 27127f3a6c64..a40b209c6ebb 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 @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.regionserver; +import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; import java.util.Collection; import java.util.Comparator; @@ -49,12 +50,16 @@ public interface StoreFileManager { * Loads the initial store files into empty StoreFileManager. * @param storeFiles The files to load. */ + @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "", + allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)") void loadFiles(List storeFiles); /** * Adds new files, either for from MemStore flush or bulk insert, into the structure. * @param sfs New store files. */ + @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "", + allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)") void insertNewFiles(Collection sfs); /** @@ -62,12 +67,16 @@ public interface StoreFileManager { * @param compactedFiles The input files for the compaction. * @param results The resulting files for the compaction. */ + @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "", + allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)") void addCompactionResults(Collection compactedFiles, Collection results); /** * Remove the compacted files * @param compactedFiles the list of compacted files */ + @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "", + allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)") void removeCompactedFiles(Collection compactedFiles); /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java index 1064b6c70547..58031288f751 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java @@ -70,10 +70,17 @@ protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum, writer.close(); } + protected final StoreFileWriter createWriter(MemStoreSnapshot snapshot, boolean alwaysIncludesTag) + throws IOException { + return store.getStoreEngine() + .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(snapshot.getCellsCount()) + .compression(store.getColumnFamilyDescriptor().getCompressionType()).isCompaction(false) + .includeMVCCReadpoint(true).includesTag(alwaysIncludesTag || snapshot.isTagsPresent()) + .shouldDropBehind(false)); + } /** * Creates the scanner for flushing snapshot. Also calls coprocessors. - * @param snapshotScanners * @return The scanner; null if coprocessor is canceling the flush. */ protected final InternalScanner createScanner(List snapshotScanners, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java index 454b244fb512..10a9330f8326 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java @@ -20,10 +20,13 @@ import java.io.IOException; import java.util.Collection; +import java.util.List; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; - +import java.util.function.Predicate; +import java.util.function.ToLongFunction; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; @@ -42,10 +45,13 @@ * Utility functions for region server storage layer. */ @InterfaceAudience.Private -public class StoreUtils { +public final class StoreUtils { private static final Logger LOG = LoggerFactory.getLogger(StoreUtils.class); + private StoreUtils() { + } + /** * Creates a deterministic hash code for store file collection. */ @@ -171,4 +177,31 @@ public static Configuration createStoreConfiguration(Configuration conf, TableDe return new CompoundConfiguration().add(conf).addBytesMap(td.getValues()) .addStringMap(cfd.getConfiguration()).addBytesMap(cfd.getValues()); } + + public static List toStoreFileInfo(Collection storefiles) { + return storefiles.stream().map(HStoreFile::getFileInfo).collect(Collectors.toList()); + } + + public static long getTotalUncompressedBytes(List files) { + return files.stream() + .mapToLong(file -> getStorefileFieldSize(file, StoreFileReader::getTotalUncompressedBytes)) + .sum(); + } + + public static long getStorefilesSize(Collection files, + Predicate predicate) { + return files.stream().filter(predicate) + .mapToLong(file -> getStorefileFieldSize(file, StoreFileReader::length)).sum(); + } + + public static long getStorefileFieldSize(HStoreFile file, ToLongFunction f) { + if (file == null) { + return 0L; + } + StoreFileReader reader = file.getReader(); + if (reader == null) { + return 0L; + } + return f.applyAsLong(reader); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java index 14863a69a9b1..bfb3f649ff27 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java @@ -20,20 +20,19 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java index 1560aef5f6b3..f8183b7645a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java @@ -70,7 +70,7 @@ public List flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushSeqNum StripeMultiFileWriter mw = null; try { mw = req.createWriter(); // Writer according to the policy. - StripeMultiFileWriter.WriterFactory factory = createWriterFactory(cellsCount); + StripeMultiFileWriter.WriterFactory factory = createWriterFactory(snapshot); StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner)scanner : null; mw.init(storeScanner, factory); @@ -98,13 +98,12 @@ public List flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushSeqNum return result; } - private StripeMultiFileWriter.WriterFactory createWriterFactory(final long kvCount) { + private StripeMultiFileWriter.WriterFactory createWriterFactory(MemStoreSnapshot snapshot) { return new StripeMultiFileWriter.WriterFactory() { @Override public StoreFileWriter createWriter() throws IOException { - StoreFileWriter writer = store.createWriterInTmp(kvCount, - store.getColumnFamilyDescriptor().getCompressionType(), false, true, true, false); - return writer; + // XXX: it used to always pass true for includesTag, re-consider? + return StripeStoreFlusher.this.createWriter(snapshot, true); } }; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java index 42841bfee531..533be176e7a7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java @@ -51,13 +51,14 @@ protected void initMultiWriter(AbstractMultiFileWriter writer, InternalScanner s WriterFactory writerFactory = new WriterFactory() { @Override public StoreFileWriter createWriter() throws IOException { - return createTmpWriter(fd, shouldDropBehind, major); + return AbstractMultiOutputCompactor.this.createWriter(fd, shouldDropBehind, major); } @Override public StoreFileWriter createWriterWithStoragePolicy(String fileStoragePolicy) - throws IOException { - return createTmpWriter(fd, shouldDropBehind, fileStoragePolicy, major); + throws IOException { + return AbstractMultiOutputCompactor.this.createWriter(fd, shouldDropBehind, + fileStoragePolicy, major); } }; // Prepare multi-writer, and perform the compaction using scanner and writer. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index e524f7dfd5fd..47ef0f290251 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -28,7 +28,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -39,6 +38,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileInfo; import org.apache.hadoop.hbase.regionserver.CellSink; +import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -61,6 +61,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.io.Closeables; /** @@ -261,29 +262,32 @@ public InternalScanner createScanner(ScanInfo scanInfo, List s } }; + protected final CreateStoreFileWriterParams createParams(FileDetails fd, boolean shouldDropBehind, + boolean major) { + return CreateStoreFileWriterParams.create().maxKeyCount(fd.maxKeyCount) + .compression(major ? majorCompactionCompression : minorCompactionCompression) + .isCompaction(true).includeMVCCReadpoint(fd.maxMVCCReadpoint > 0) + .includesTag(fd.maxTagsLength > 0).shouldDropBehind(shouldDropBehind) + .totalCompactedFilesSize(fd.totalCompactedFilesSize); + } + /** - * Creates a writer for a new file in a temporary directory. + * Creates a writer for a new file. * @param fd The file details. - * @return Writer for a new StoreFile in the tmp dir. + * @return Writer for a new StoreFile * @throws IOException if creation failed */ - protected final StoreFileWriter createTmpWriter(FileDetails fd, boolean shouldDropBehind, boolean major) - throws IOException { + protected final StoreFileWriter createWriter(FileDetails fd, boolean shouldDropBehind, + boolean major) throws IOException { // When all MVCC readpoints are 0, don't write them. // See HBASE-8166, HBASE-12600, and HBASE-13389. - return store.createWriterInTmp(fd.maxKeyCount, - major ? majorCompactionCompression : minorCompactionCompression, - true, fd.maxMVCCReadpoint > 0, - fd.maxTagsLength > 0, shouldDropBehind, fd.totalCompactedFilesSize, - HConstants.EMPTY_STRING); + return store.getStoreEngine().createWriter(createParams(fd, shouldDropBehind, major)); } - protected final StoreFileWriter createTmpWriter(FileDetails fd, boolean shouldDropBehind, - String fileStoragePolicy, boolean major) throws IOException { - return store.createWriterInTmp(fd.maxKeyCount, - major ? majorCompactionCompression : minorCompactionCompression, - true, fd.maxMVCCReadpoint > 0, - fd.maxTagsLength > 0, shouldDropBehind, fd.totalCompactedFilesSize, fileStoragePolicy); + protected final StoreFileWriter createWriter(FileDetails fd, boolean shouldDropBehind, + String fileStoragePolicy, boolean major) throws IOException { + return store.getStoreEngine() + .createWriter(createParams(fd, shouldDropBehind, major).fileStoragePolicy(fileStoragePolicy)); } private ScanInfo preCompactScannerOpen(CompactionRequestImpl request, ScanType scanType, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java index 49d3e8ee01e7..afa2429cb6e8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java @@ -45,14 +45,14 @@ public DefaultCompactor(Configuration conf, HStore store) { } private final CellSinkFactory writerFactory = - new CellSinkFactory() { - @Override - public StoreFileWriter createWriter(InternalScanner scanner, - org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd, - boolean shouldDropBehind, boolean major) throws IOException { - return createTmpWriter(fd, shouldDropBehind, major); - } - }; + new CellSinkFactory() { + @Override + public StoreFileWriter createWriter(InternalScanner scanner, + org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd, + boolean shouldDropBehind, boolean major) throws IOException { + return DefaultCompactor.this.createWriter(fd, shouldDropBehind, major); + } + }; /** * Do a minor/major compaction on an explicit set of storefiles from a Store. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java new file mode 100644 index 000000000000..d4c9a868eb48 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java @@ -0,0 +1,61 @@ +/** + * 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.storefiletracker; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * The default implementation for store file tracker, where we do not persist the store file list, + * and use listing when loading store files. + */ +@InterfaceAudience.Private +class DefaultStoreFileTracker extends StoreFileTrackerBase { + + public DefaultStoreFileTracker(Configuration conf, TableName tableName, boolean isPrimaryReplica, + StoreContext ctx) { + super(conf, tableName, isPrimaryReplica, ctx); + } + + @Override + public List load() throws IOException { + return ctx.getRegionFileSystem().getStoreFiles(ctx.getFamily().getNameAsString()); + } + + @Override + public boolean requireWritingToTmpDirFirst() { + return true; + } + + @Override + protected void doAddNewStoreFiles(Collection newFiles) throws IOException { + // NOOP + } + + @Override + protected void doAddCompactionResults(Collection compactedFiles, + Collection newFiles) throws IOException { + // NOOP + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java new file mode 100644 index 000000000000..aadedc8ef727 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java @@ -0,0 +1,75 @@ +/** + * 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.storefiletracker; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * An interface to define how we track the store files for a give store. + *

+ * In the old time, we will write store to a tmp directory first, and then rename it to the actual + * data file. And once a store file is under data directory, we will consider it as 'committed'. And + * we need to do listing when loading store files. + *

+ * When cloud age is coming, now we want to store the store files on object storage, where rename + * and list are not as cheap as on HDFS, especially rename. Although introducing a metadata + * management layer for object storage could solve the problem, but we still want HBase to run on + * pure object storage, so here we introduce this interface to abstract how we track the store + * files. For the old implementation, we just persist nothing here, and do listing to load store + * files. When running on object storage, we could persist the store file list in a system region, + * or in a file on the object storage, to make it possible to write directly into the data directory + * to avoid renaming, and also avoid listing when loading store files. + *

+ * The implementation requires to be thread safe as flush and compaction may occur as the same time, + * and we could also do multiple compactions at the same time. As the implementation may choose to + * persist the store file list to external storage, which could be slow, it is the duty for the + * callers to not call it inside a lock which may block normal read/write requests. + */ +@InterfaceAudience.Private +public interface StoreFileTracker { + + /** + * Load the store files list when opening a region. + */ + List load() throws IOException; + + /** + * Add new store files. + *

+ * Used for flush and bulk load. + */ + void add(Collection newFiles) throws IOException; + + /** + * Add new store files and remove compacted store files after compaction. + */ + void replace(Collection compactedFiles, Collection newFiles) + throws IOException; + + /** + * Create a writer for writing new store files. + * @return Writer for a new StoreFile + */ + StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java new file mode 100644 index 000000000000..2451f45bdc7b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -0,0 +1,178 @@ +/** + * 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.storefiletracker; + +import java.io.IOException; +import java.util.Collection; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hadoop.hbase.io.crypto.Encryption; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; +import org.apache.hadoop.hbase.regionserver.StoreUtils; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Base class for all store file tracker. + *

+ * Mainly used to place the common logic to skip persistent for secondary replicas. + */ +@InterfaceAudience.Private +abstract class StoreFileTrackerBase implements StoreFileTracker { + + private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerBase.class); + + protected final Configuration conf; + + protected final TableName tableName; + + protected final boolean isPrimaryReplica; + + protected final StoreContext ctx; + + private volatile boolean cacheOnWriteLogged; + + protected StoreFileTrackerBase(Configuration conf, TableName tableName, boolean isPrimaryReplica, + StoreContext ctx) { + this.conf = conf; + this.tableName = tableName; + this.isPrimaryReplica = isPrimaryReplica; + this.ctx = ctx; + } + + @Override + public final void add(Collection newFiles) throws IOException { + if (isPrimaryReplica) { + doAddNewStoreFiles(newFiles); + } + } + + @Override + public final void replace(Collection compactedFiles, + Collection newFiles) throws IOException { + if (isPrimaryReplica) { + doAddCompactionResults(compactedFiles, newFiles); + } + } + + private HFileContext createFileContext(Compression.Algorithm compression, + boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context encryptionContext) { + if (compression == null) { + compression = HFile.DEFAULT_COMPRESSION_ALGORITHM; + } + ColumnFamilyDescriptor family = ctx.getFamily(); + HFileContext hFileContext = new HFileContextBuilder().withIncludesMvcc(includeMVCCReadpoint) + .withIncludesTags(includesTag).withCompression(compression) + .withCompressTags(family.isCompressTags()).withChecksumType(StoreUtils.getChecksumType(conf)) + .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf)) + .withBlockSize(family.getBlocksize()).withHBaseCheckSum(true) + .withDataBlockEncoding(family.getDataBlockEncoding()).withEncryptionContext(encryptionContext) + .withCreateTime(EnvironmentEdgeManager.currentTime()).withColumnFamily(family.getName()) + .withTableName(tableName.getName()).withCellComparator(ctx.getComparator()).build(); + return hFileContext; + } + + @Override + public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) + throws IOException { + if (!isPrimaryReplica) { + throw new IllegalStateException("Should not call create writer on secondary replicas"); + } + // creating new cache config for each new writer + final CacheConfig cacheConf = ctx.getCacheConf(); + final CacheConfig writerCacheConf = new CacheConfig(cacheConf); + long totalCompactedFilesSize = params.totalCompactedFilesSize(); + if (params.isCompaction()) { + // Don't cache data on write on compactions, unless specifically configured to do so + // Cache only when total file size remains lower than configured threshold + final boolean cacheCompactedBlocksOnWrite = cacheConf.shouldCacheCompactedBlocksOnWrite(); + // if data blocks are to be cached on write + // during compaction, we should forcefully + // cache index and bloom blocks as well + if (cacheCompactedBlocksOnWrite && + totalCompactedFilesSize <= cacheConf.getCacheCompactedBlocksOnWriteThreshold()) { + writerCacheConf.enableCacheOnWrite(); + if (!cacheOnWriteLogged) { + LOG.info("For {} , cacheCompactedBlocksOnWrite is true, hence enabled " + + "cacheOnWrite for Data blocks, Index blocks and Bloom filter blocks", this); + cacheOnWriteLogged = true; + } + } else { + writerCacheConf.setCacheDataOnWrite(false); + if (totalCompactedFilesSize > cacheConf.getCacheCompactedBlocksOnWriteThreshold()) { + // checking condition once again for logging + LOG.debug( + "For {}, setting cacheCompactedBlocksOnWrite as false as total size of compacted " + + "files - {}, is greater than cacheCompactedBlocksOnWriteThreshold - {}", + this, totalCompactedFilesSize, cacheConf.getCacheCompactedBlocksOnWriteThreshold()); + } + } + } else { + final boolean shouldCacheDataOnWrite = cacheConf.shouldCacheDataOnWrite(); + if (shouldCacheDataOnWrite) { + writerCacheConf.enableCacheOnWrite(); + if (!cacheOnWriteLogged) { + LOG.info("For {} , cacheDataOnWrite is true, hence enabled cacheOnWrite for " + + "Index blocks and Bloom filter blocks", this); + cacheOnWriteLogged = true; + } + } + } + Encryption.Context encryptionContext = ctx.getEncryptionContext(); + HFileContext hFileContext = createFileContext(params.compression(), + params.includeMVCCReadpoint(), params.includesTag(), encryptionContext); + Path outputDir; + if (requireWritingToTmpDirFirst()) { + outputDir = + new Path(ctx.getRegionFileSystem().getTempDir(), ctx.getFamily().getNameAsString()); + } else { + throw new UnsupportedOperationException("not supported yet"); + } + StoreFileWriter.Builder builder = + new StoreFileWriter.Builder(conf, writerCacheConf, ctx.getRegionFileSystem().getFileSystem()) + .withOutputDir(outputDir).withBloomType(ctx.getBloomFilterType()) + .withMaxKeyCount(params.maxKeyCount()).withFavoredNodes(ctx.getFavoredNodes()) + .withFileContext(hFileContext).withShouldDropCacheBehind(params.shouldDropBehind()) + .withCompactedFilesSupplier(ctx.getCompactedFilesSupplier()) + .withFileStoragePolicy(params.fileStoragePolicy()); + return builder.build(); + } + + /** + * Whether the implementation of this tracker requires you to write to temp directory first, i.e, + * does not allow broken store files under the actual data directory. + */ + protected abstract boolean requireWritingToTmpDirFirst(); + + protected abstract void doAddNewStoreFiles(Collection newFiles) throws IOException; + + protected abstract void doAddCompactionResults(Collection compactedFiles, + Collection newFiles) throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java new file mode 100644 index 000000000000..4f7231bc3b9a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -0,0 +1,35 @@ +/** + * 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.storefiletracker; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Factory method for creating store file tracker. + */ +@InterfaceAudience.Private +public final class StoreFileTrackerFactory { + + public static StoreFileTracker create(Configuration conf, TableName tableName, + boolean isPrimaryReplica, StoreContext ctx) { + return new DefaultStoreFileTracker(conf, tableName, isPrimaryReplica, ctx); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java index 22ec6cb89ec4..291b909f69bf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java @@ -105,7 +105,6 @@ Set getStoresRequiringCompaction(Set requestedStores, long times boolean shouldCFBeCompacted(HRegionFileSystem fileSystem, String family, long ts) throws IOException { - // do we have any store files? Collection storeFiles = fileSystem.getStoreFiles(family); if (storeFiles == null) { 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 3c2bc3f0cb12..9314d7e9827f 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 @@ -211,11 +211,13 @@ protected BlockCompactionsInCompletionHStore(HRegion region, ColumnFamilyDescrip @Override protected void refreshStoreSizeAndTotalBytes() throws IOException { - try { - r.compactionsWaiting.countDown(); - r.compactionsBlocked.await(); - } catch (InterruptedException ex) { - throw new IOException(ex); + if (r != null) { + try { + r.compactionsWaiting.countDown(); + r.compactionsBlocked.await(); + } catch (InterruptedException ex) { + throw new IOException(ex); + } } super.refreshStoreSizeAndTotalBytes(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index 54b0d182d54c..7dbb68090946 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -215,8 +215,10 @@ public void tearDown() throws IOException { @Test public void testCacheOnWriteInSchema() throws IOException { // Write some random data into the store - StoreFileWriter writer = store.createWriterInTmp(Integer.MAX_VALUE, - HFile.DEFAULT_COMPRESSION_ALGORITHM, false, true, false, false); + StoreFileWriter writer = store.getStoreEngine() + .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(Integer.MAX_VALUE) + .compression(HFile.DEFAULT_COMPRESSION_ALGORITHM).isCompaction(false) + .includeMVCCReadpoint(true).includesTag(false).shouldDropBehind(false)); writeStoreFile(writer); writer.close(); // Verify the block types of interest were cached on write diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java index e832c47aac81..3784876a59f3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java @@ -65,9 +65,12 @@ public void testCustomParts() throws Exception { DummyCompactionPolicy.class.getName()); conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY, DummyStoreFlusher.class.getName()); + HRegion mockRegion = Mockito.mock(HRegion.class); HStore mockStore = Mockito.mock(HStore.class); Mockito.when(mockStore.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO); - StoreEngine se = StoreEngine.create(mockStore, conf, CellComparatorImpl.COMPARATOR); + Mockito.when(mockStore.getHRegion()).thenReturn(mockRegion); + StoreEngine se = + StoreEngine.create(mockStore, conf, CellComparatorImpl.COMPARATOR); Assert.assertTrue(se instanceof DefaultStoreEngine); Assert.assertTrue(se.getCompactionPolicy() instanceof DummyCompactionPolicy); Assert.assertTrue(se.getStoreFlusher() instanceof DummyStoreFlusher); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 1d302d482a1c..8bcbf10a2b97 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5761,7 +5761,7 @@ public void testCompactionFromPrimary() throws IOException { Collection storeFiles = primaryRegion.getStore(families[0]).getStorefiles(); primaryRegion.getRegionFileSystem().removeStoreFiles(Bytes.toString(families[0]), storeFiles); Collection storeFileInfos = primaryRegion.getRegionFileSystem() - .getStoreFiles(families[0]); + .getStoreFiles(Bytes.toString(families[0])); Assert.assertTrue(storeFileInfos == null || storeFileInfos.isEmpty()); verifyData(secondaryRegion, 0, 1000, cq, families); @@ -7665,7 +7665,7 @@ protected List doCompaction(CompactionRequestImpl cr, getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true; for (Path newFile : newFiles) { // Create storefile around what we wrote with a reader on it. - HStoreFile sf = createStoreFileAndReader(newFile); + HStoreFile sf = storeEngine.createStoreFileAndReader(newFile); sf.closeStoreFile(evictOnClose); sfs.add(sf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index 4e373a823c56..c36e97541aff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -53,8 +53,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.IntBinaryOperator; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -313,7 +313,7 @@ public Object run() throws Exception { /** * Verify that compression and data block encoding are respected by the - * Store.createWriterInTmp() method, used on store flush. + * createWriter method, used on store flush. */ @Test public void testCreateWriter() throws Exception { @@ -325,9 +325,11 @@ public void testCreateWriter() throws Exception { .build(); init(name.getMethodName(), conf, hcd); - // Test createWriterInTmp() - StoreFileWriter writer = - store.createWriterInTmp(4, hcd.getCompressionType(), false, true, false, false); + // Test createWriter + StoreFileWriter writer = store.getStoreEngine() + .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(4) + .compression(hcd.getCompressionType()).isCompaction(false).includeMVCCReadpoint(true) + .includesTag(false).shouldDropBehind(false)); Path path = writer.getPath(); writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1))); writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2))); @@ -1024,19 +1026,19 @@ public void testRefreshStoreFilesNotChanged() throws IOException { // add one more file addStoreFile(); - HStore spiedStore = spy(store); + StoreEngine spiedStoreEngine = spy(store.getStoreEngine()); // call first time after files changed - spiedStore.refreshStoreFiles(); + spiedStoreEngine.refreshStoreFiles(); assertEquals(2, this.store.getStorefilesCount()); - verify(spiedStore, times(1)).replaceStoreFiles(any(), any()); + verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any()); // call second time - spiedStore.refreshStoreFiles(); + spiedStoreEngine.refreshStoreFiles(); // ensure that replaceStoreFiles is not called, i.e, the times does not change, if files are not // refreshed, - verify(spiedStore, times(1)).replaceStoreFiles(any(), any()); + verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any()); } private long countMemStoreScanner(StoreScanner scanner) { @@ -1647,7 +1649,7 @@ public void testSwitchingPreadtoStreamParallelyWithCompactionDischarger() throws // Do compaction MyThread thread = new MyThread(storeScanner); thread.start(); - store.replaceStoreFiles(actualStorefiles, actualStorefiles1); + store.replaceStoreFiles(actualStorefiles, actualStorefiles1, false); thread.join(); KeyValueHeap heap2 = thread.getHeap(); assertFalse(heap.equals(heap2)); @@ -1713,8 +1715,10 @@ public void testSpaceQuotaChangeAfterReplacement() throws IOException { @Test public void testHFileContextSetWithCFAndTable() throws Exception { init(this.name.getMethodName()); - StoreFileWriter writer = store.createWriterInTmp(10000L, - Compression.Algorithm.NONE, false, true, false, true); + StoreFileWriter writer = store.getStoreEngine() + .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(10000L) + .compression(Compression.Algorithm.NONE).isCompaction(true).includeMVCCReadpoint(true) + .includesTag(false).shouldDropBehind(true)); HFileContext hFileContext = writer.getHFileWriter().getFileContext(); assertArrayEquals(family, hFileContext.getColumnFamily()); assertArrayEquals(table, hFileContext.getTableName()); @@ -3054,7 +3058,8 @@ protected void doClearSnapShot() { int currentCount = clearSnapshotCounter.incrementAndGet(); if (currentCount == 1) { try { - if (store.lock.isWriteLockedByCurrentThread()) { + if (((ReentrantReadWriteLock) store.getStoreEngine().getLock()) + .isWriteLockedByCurrentThread()) { shouldWait = false; } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java index 60f65be66c47..1dbf37aa8063 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java @@ -245,7 +245,7 @@ public void testCleanMergeReference() throws Exception { TEST_UTIL.getConfiguration(), fs, tabledir, mergedRegionInfo); int count = 0; for(ColumnFamilyDescriptor colFamily : columnFamilies) { - count += hrfs.getStoreFiles(colFamily.getName()).size(); + count += hrfs.getStoreFiles(colFamily.getNameAsString()).size(); } ADMIN.compactRegion(mergedRegionInfo.getRegionName()); // clean up the merged region store files @@ -254,7 +254,7 @@ public void testCleanMergeReference() throws Exception { int newcount = 0; while (EnvironmentEdgeManager.currentTime() < timeout) { for(ColumnFamilyDescriptor colFamily : columnFamilies) { - newcount += hrfs.getStoreFiles(colFamily.getName()).size(); + newcount += hrfs.getStoreFiles(colFamily.getNameAsString()).size(); } if(newcount > count) { break; @@ -273,7 +273,7 @@ public void testCleanMergeReference() throws Exception { while (EnvironmentEdgeManager.currentTime() < timeout) { int newcount1 = 0; for(ColumnFamilyDescriptor colFamily : columnFamilies) { - newcount1 += hrfs.getStoreFiles(colFamily.getName()).size(); + newcount1 += hrfs.getStoreFiles(colFamily.getNameAsString()).size(); } if(newcount1 <= 1) { break; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java index 9141327d26d4..2fab050446ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -98,7 +97,7 @@ static class FailingHRegionFileSystem extends HRegionFileSystem { } @Override - public Collection getStoreFiles(String familyName) throws IOException { + public List getStoreFiles(String familyName) throws IOException { if (fail) { throw new IOException("simulating FS failure"); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java index c2b784914d12..79a970571f09 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java @@ -29,7 +29,6 @@ import java.util.Random; import java.util.TreeSet; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -128,13 +127,12 @@ public void testScannerCloseAndUpdateReadersWithMemstoreScanner() throws Excepti p.addColumn(fam, Bytes.toBytes("q1"), Bytes.toBytes("val")); region.put(p); HStore store = region.getStore(fam); - ReentrantReadWriteLock lock = store.lock; // use the lock to manually get a new memstore scanner. this is what // HStore#notifyChangedReadersObservers does under the lock.(lock is not needed here //since it is just a testcase). - lock.readLock().lock(); + store.getStoreEngine().readLock(); final List memScanners = store.memstore.getScanners(Long.MAX_VALUE); - lock.readLock().unlock(); + store.getStoreEngine().readUnlock(); Thread closeThread = new Thread() { public void run() { // close should be completed diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java index f5330f6faa42..eb0b1c1ca694 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java @@ -118,8 +118,10 @@ private static HStoreFile createFile() throws Exception { } private static TestStoreEngine createEngine(Configuration conf) throws Exception { + HRegion region = mock(HRegion.class); HStore store = mock(HStore.class); when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO); + when(store.getHRegion()).thenReturn(region); CellComparatorImpl kvComparator = mock(CellComparatorImpl.class); return (TestStoreEngine) StoreEngine.create(store, conf, kvComparator); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java index d7b7ba760156..0ea82c57be1a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java @@ -22,9 +22,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -45,11 +42,13 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner; @@ -110,10 +109,9 @@ private DateTieredCompactor createCompactor(StoreFileWritersCapture writers, when(store.areWritesEnabled()).thenReturn(true); when(store.getFileSystem()).thenReturn(mock(FileSystem.class)); when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.newBuilder(TABLE_NAME).build()); - when(store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers); - when(store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyString())).thenAnswer(writers); + StoreEngine storeEngine = mock(StoreEngine.class); + when(storeEngine.createWriter(any(CreateStoreFileWriterParams.class))).thenAnswer(writers); + when(store.getStoreEngine()).thenReturn(storeEngine); when(store.getComparator()).thenReturn(CellComparatorImpl.COMPARATOR); OptionalLong maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles); when(store.getMaxSequenceId()).thenReturn(maxSequenceId); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java index 2ba15d1be18e..bdab20ead8a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java @@ -30,7 +30,6 @@ import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.isNull; @@ -60,6 +59,7 @@ import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; +import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter; @@ -864,12 +865,9 @@ private StripeCompactor createCompactor() throws Exception { when(info.getRegionNameAsString()).thenReturn("testRegion"); when(store.getColumnFamilyDescriptor()).thenReturn(familyDescriptor); when(store.getRegionInfo()).thenReturn(info); - when( - store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers); - when( - store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyString())).thenAnswer(writers); + StoreEngine storeEngine = mock(StoreEngine.class); + when(storeEngine.createWriter(any(CreateStoreFileWriterParams.class))).thenAnswer(writers); + when(store.getStoreEngine()).thenReturn(storeEngine); Configuration conf = HBaseConfiguration.create(); conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java index e49174e6afe3..ae59c74bad95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java @@ -21,9 +21,6 @@ import static org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.createDummyRequest; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -43,10 +40,12 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner; import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture; @@ -209,10 +208,9 @@ private StripeCompactor createCompactor(StoreFileWritersCapture writers, KeyValu when(store.areWritesEnabled()).thenReturn(true); when(store.getFileSystem()).thenReturn(mock(FileSystem.class)); when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.newBuilder(TABLE_NAME).build()); - when(store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers); - when(store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyString())).thenAnswer(writers); + StoreEngine storeEngine = mock(StoreEngine.class); + when(storeEngine.createWriter(any(CreateStoreFileWriterParams.class))).thenAnswer(writers); + when(store.getStoreEngine()).thenReturn(storeEngine); when(store.getComparator()).thenReturn(CellComparatorImpl.COMPARATOR); return new StripeCompactor(conf, store) { From 2eceabd19bea85e53dfbb71a130db796c304b5c8 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 26 Aug 2021 18:51:12 +0800 Subject: [PATCH 02/26] HBASE-25988 Store the store file list by a file (#3578) Signed-off-by: Wellington Chevreuil --- .../server/region/StoreFileTracker.proto | 36 +++++ .../hbase/regionserver/StoreContext.java | 5 + .../hbase/regionserver/StoreEngine.java | 8 +- .../DefaultStoreFileTracker.java | 5 +- .../FileBasedStoreFileTracker.java | 142 ++++++++++++++++++ .../storefiletracker/StoreFileListFile.java | 142 ++++++++++++++++++ .../StoreFileTrackerBase.java | 11 +- .../StoreFileTrackerFactory.java | 12 +- ...stRegionWithFileBasedStoreFileTracker.java | 109 ++++++++++++++ 9 files changed, 451 insertions(+), 19 deletions(-) create mode 100644 hbase-protocol-shaded/src/main/protobuf/server/region/StoreFileTracker.proto create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java diff --git a/hbase-protocol-shaded/src/main/protobuf/server/region/StoreFileTracker.proto b/hbase-protocol-shaded/src/main/protobuf/server/region/StoreFileTracker.proto new file mode 100644 index 000000000000..2a269ea4ac4e --- /dev/null +++ b/hbase-protocol-shaded/src/main/protobuf/server/region/StoreFileTracker.proto @@ -0,0 +1,36 @@ +/** + * 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. + */ +syntax = "proto2"; +// This file contains protocol buffers that are used for store file tracker. +package hbase.pb; + +option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated"; +option java_outer_classname = "StoreFileTrackerProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +message StoreFileEntry { + required string name = 1; + required uint64 size = 2; +} + +message StoreFileList { + required uint64 timestamp = 1; + repeated StoreFileEntry store_file = 2; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java index 2a9f96859361..588f8f4027a6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java @@ -22,6 +22,7 @@ import java.util.function.Supplier; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.io.HeapSize; @@ -109,6 +110,10 @@ public RegionCoprocessorHost getCoprocessorHost() { return coprocessorHost; } + public TableName getTableName() { + return getRegionInfo().getTable(); + } + public RegionInfo getRegionInfo() { return regionFileSystem.getRegionInfo(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java index 4033c335d52c..04867295c3ae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java @@ -173,9 +173,9 @@ public StoreFlusher getStoreFlusher() { return this.storeFlusher; } - private StoreFileTracker createStoreFileTracker(HStore store) { - return StoreFileTrackerFactory.create(store.conf, store.getRegionInfo().getTable(), - store.isPrimaryReplicaStore(), store.getStoreContext()); + private StoreFileTracker createStoreFileTracker(Configuration conf, HStore store) { + return StoreFileTrackerFactory.create(conf, store.isPrimaryReplicaStore(), + store.getStoreContext()); } /** @@ -206,7 +206,7 @@ protected final void createComponentsOnce(Configuration conf, HStore store, this.ctx = store.getStoreContext(); this.coprocessorHost = store.getHRegion().getCoprocessorHost(); this.openStoreFileThreadPoolCreator = store.getHRegion()::getStoreFileOpenAndCloseThreadPool; - this.storeFileTracker = createStoreFileTracker(store); + this.storeFileTracker = createStoreFileTracker(conf, store); assert compactor != null && compactionPolicy != null && storeFileManager != null && storeFlusher != null && storeFileTracker != null; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java index d4c9a868eb48..fa044818336f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java @@ -21,7 +21,6 @@ import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.yetus.audience.InterfaceAudience; @@ -33,9 +32,9 @@ @InterfaceAudience.Private class DefaultStoreFileTracker extends StoreFileTrackerBase { - public DefaultStoreFileTracker(Configuration conf, TableName tableName, boolean isPrimaryReplica, + public DefaultStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { - super(conf, tableName, isPrimaryReplica, ctx); + super(conf, isPrimaryReplica, ctx); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java new file mode 100644 index 000000000000..de28b0eb9996 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java @@ -0,0 +1,142 @@ +/** + * 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.storefiletracker; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.StoreFileTrackerProtos.StoreFileEntry; +import org.apache.hadoop.hbase.shaded.protobuf.generated.StoreFileTrackerProtos.StoreFileList; + +/** + * A file based store file tracker. + *

+ * For this tracking way, the store file list will be persistent into a file, so we can write the + * new store files directly to the final data directory, as we will not load the broken files. This + * will greatly reduce the time for flush and compaction on some object storages as a rename is + * actual a copy on them. And it also avoid listing when loading store file list, which could also + * speed up the loading of store files as listing is also not a fast operation on most object + * storages. + */ +@InterfaceAudience.Private +public class FileBasedStoreFileTracker extends StoreFileTrackerBase { + + private final StoreFileListFile backedFile; + + private final Map storefiles = new HashMap<>(); + + public FileBasedStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { + super(conf, isPrimaryReplica, ctx); + backedFile = new StoreFileListFile(ctx); + } + + @Override + public List load() throws IOException { + StoreFileList list = backedFile.load(); + if (list == null) { + return Collections.emptyList(); + } + FileSystem fs = ctx.getRegionFileSystem().getFileSystem(); + List infos = new ArrayList<>(); + for (StoreFileEntry entry : list.getStoreFileList()) { + infos.add(ServerRegionReplicaUtil.getStoreFileInfo(conf, fs, ctx.getRegionInfo(), + ctx.getRegionFileSystem().getRegionInfoForFS(), ctx.getFamily().getNameAsString(), + new Path(ctx.getFamilyStoreDirectoryPath(), entry.getName()))); + } + // In general, for primary replica, the load method should only be called once when + // initialization, so we do not need synchronized here. And for secondary replicas, though the + // load method could be called multiple times, we will never call other methods so no + // synchronized is also fine. + // But we have a refreshStoreFiles method in the Region interface, which can be called by CPs, + // and we have a RefreshHFilesEndpoint example to expose the refreshStoreFiles method as RPC, so + // for safety, let's still keep the synchronized here. + synchronized (storefiles) { + for (StoreFileInfo info : infos) { + storefiles.put(info.getPath().getName(), info); + } + } + return infos; + } + + @Override + protected boolean requireWritingToTmpDirFirst() { + return false; + } + + private StoreFileEntry toStoreFileEntry(StoreFileInfo info) { + return StoreFileEntry.newBuilder().setName(info.getPath().getName()).setSize(info.getSize()) + .build(); + } + + @Override + protected void doAddNewStoreFiles(Collection newFiles) throws IOException { + synchronized (storefiles) { + StoreFileList.Builder builder = StoreFileList.newBuilder(); + for (StoreFileInfo info : storefiles.values()) { + builder.addStoreFile(toStoreFileEntry(info)); + } + for (StoreFileInfo info : newFiles) { + builder.addStoreFile(toStoreFileEntry(info)); + } + backedFile.update(builder); + for (StoreFileInfo info : newFiles) { + storefiles.put(info.getPath().getName(), info); + } + } + } + + @Override + protected void doAddCompactionResults(Collection compactedFiles, + Collection newFiles) throws IOException { + Set compactedFileNames = + compactedFiles.stream().map(info -> info.getPath().getName()).collect(Collectors.toSet()); + synchronized (storefiles) { + StoreFileList.Builder builder = StoreFileList.newBuilder(); + storefiles.forEach((name, info) -> { + if (compactedFileNames.contains(name)) { + return; + } + builder.addStoreFile(toStoreFileEntry(info)); + }); + for (StoreFileInfo info : newFiles) { + builder.addStoreFile(toStoreFileEntry(info)); + } + backedFile.update(builder); + for (String name : compactedFileNames) { + storefiles.remove(name); + } + for (StoreFileInfo info : newFiles) { + storefiles.put(info.getPath().getName(), info); + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java new file mode 100644 index 000000000000..c778bfc51deb --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java @@ -0,0 +1,142 @@ +/** + * 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.storefiletracker; + +import java.io.FileNotFoundException; +import java.io.IOException; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; +import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.StoreFileTrackerProtos.StoreFileList; + +/** + * To fully avoid listing, here we use two files for tracking. When loading, we will try to read + * both the two files, if only one exists, we will trust this one, if both exist, we will compare + * the timestamp to see which one is newer and trust that one. And we will record in memory that + * which one is trusted by us, and when we need to update the store file list, we will write to the + * other file. + *

+ * So in this way, we could avoid listing when we want to load the store file list file. + */ +@InterfaceAudience.Private +class StoreFileListFile { + + private static final Logger LOG = LoggerFactory.getLogger(StoreFileListFile.class); + + private static final String TRACK_FILE_DIR = ".filelist"; + + private static final String TRACK_FILE = "f1"; + + private static final String TRACK_FILE_ROTATE = "f2"; + + private final StoreContext ctx; + + private final Path trackFileDir; + + private final Path[] trackFiles = new Path[2]; + + // this is used to make sure that we do not go backwards + private long prevTimestamp = -1; + + private int nextTrackFile = -1; + + StoreFileListFile(StoreContext ctx) { + this.ctx = ctx; + trackFileDir = new Path(ctx.getFamilyStoreDirectoryPath(), TRACK_FILE_DIR); + trackFiles[0] = new Path(trackFileDir, TRACK_FILE); + trackFiles[1] = new Path(trackFileDir, TRACK_FILE_ROTATE); + } + + private StoreFileList load(Path path) throws IOException { + FileSystem fs = ctx.getRegionFileSystem().getFileSystem(); + byte[] bytes; + try (FSDataInputStream in = fs.open(path)) { + bytes = ByteStreams.toByteArray(in); + } + // Read all the bytes and then parse it, so we will only throw InvalidProtocolBufferException + // here. This is very important for upper layer to determine whether this is the normal case, + // where the file does not exist or is incomplete. If there is another type of exception, the + // upper layer should throw it out instead of just ignoring it, otherwise it will lead to data + // loss. + return StoreFileList.parseFrom(bytes); + } + + private int select(StoreFileList[] lists) { + if (lists[0] == null) { + return 1; + } + if (lists[1] == null) { + return 0; + } + return lists[0].getTimestamp() >= lists[1].getTimestamp() ? 0 : 1; + } + + StoreFileList load() throws IOException { + StoreFileList[] lists = new StoreFileList[2]; + for (int i = 0; i < 2; i++) { + try { + lists[i] = load(trackFiles[i]); + } catch (FileNotFoundException | InvalidProtocolBufferException e) { + // this is normal case, so use info and do not log stacktrace + LOG.info("Failed to load track file {}: {}", trackFiles[i], e); + } + } + int winnerIndex = select(lists); + if (lists[winnerIndex] != null) { + nextTrackFile = 1 - winnerIndex; + prevTimestamp = lists[winnerIndex].getTimestamp(); + } else { + nextTrackFile = 0; + } + return lists[winnerIndex]; + } + + /** + * We will set the timestamp in this method so just pass the builder in + */ + void update(StoreFileList.Builder builder) throws IOException { + Preconditions.checkState(nextTrackFile >= 0, "should call load first before calling update"); + FileSystem fs = ctx.getRegionFileSystem().getFileSystem(); + long timestamp = Math.max(prevTimestamp + 1, EnvironmentEdgeManager.currentTime()); + try (FSDataOutputStream out = fs.create(trackFiles[nextTrackFile], true)) { + builder.setTimestamp(timestamp).build().writeTo(out); + } + // record timestamp + prevTimestamp = timestamp; + // rotate the file + nextTrackFile = 1 - nextTrackFile; + try { + fs.delete(trackFiles[nextTrackFile], false); + } catch (IOException e) { + // we will create new file with overwrite = true, so not a big deal here, only for speed up + // loading as we do not need to read this file when loading(we will hit FileNotFoundException) + LOG.debug("failed to delete old track file {}, not a big deal, just ignore", e); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java index 2451f45bdc7b..92c699278c2e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -21,7 +21,6 @@ import java.util.Collection; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.crypto.Encryption; @@ -51,18 +50,14 @@ abstract class StoreFileTrackerBase implements StoreFileTracker { protected final Configuration conf; - protected final TableName tableName; - protected final boolean isPrimaryReplica; protected final StoreContext ctx; private volatile boolean cacheOnWriteLogged; - protected StoreFileTrackerBase(Configuration conf, TableName tableName, boolean isPrimaryReplica, - StoreContext ctx) { + protected StoreFileTrackerBase(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { this.conf = conf; - this.tableName = tableName; this.isPrimaryReplica = isPrimaryReplica; this.ctx = ctx; } @@ -95,7 +90,7 @@ private HFileContext createFileContext(Compression.Algorithm compression, .withBlockSize(family.getBlocksize()).withHBaseCheckSum(true) .withDataBlockEncoding(family.getDataBlockEncoding()).withEncryptionContext(encryptionContext) .withCreateTime(EnvironmentEdgeManager.currentTime()).withColumnFamily(family.getName()) - .withTableName(tableName.getName()).withCellComparator(ctx.getComparator()).build(); + .withTableName(ctx.getTableName().getName()).withCellComparator(ctx.getComparator()).build(); return hFileContext; } @@ -153,7 +148,7 @@ public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) outputDir = new Path(ctx.getRegionFileSystem().getTempDir(), ctx.getFamily().getNameAsString()); } else { - throw new UnsupportedOperationException("not supported yet"); + outputDir = ctx.getFamilyStoreDirectoryPath(); } StoreFileWriter.Builder builder = new StoreFileWriter.Builder(conf, writerCacheConf, ctx.getRegionFileSystem().getFileSystem()) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index 4f7231bc3b9a..6cdfaf4a0fe1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; /** @@ -28,8 +28,12 @@ @InterfaceAudience.Private public final class StoreFileTrackerFactory { - public static StoreFileTracker create(Configuration conf, TableName tableName, - boolean isPrimaryReplica, StoreContext ctx) { - return new DefaultStoreFileTracker(conf, tableName, isPrimaryReplica, ctx); + public static final String TRACK_IMPL = "hbase.store.file-tracker.impl"; + + public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, + StoreContext ctx) { + Class tracker = + conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class); + return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java new file mode 100644 index 000000000000..3bc60d13fd94 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java @@ -0,0 +1,109 @@ +/** + * 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.storefiletracker; + +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestRegionWithFileBasedStoreFileTracker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionWithFileBasedStoreFileTracker.class); + + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + + private static final byte[] CF = Bytes.toBytes("cf"); + + private static final byte[] CQ = Bytes.toBytes("cq"); + + private static final TableDescriptor TD = + TableDescriptorBuilder.newBuilder(TableName.valueOf("file_based_tracker")) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build(); + + private static final RegionInfo RI = RegionInfoBuilder.newBuilder(TD.getTableName()).build(); + + @Rule + public TestName name = new TestName(); + + private HRegion region; + + @Before + public void setUp() throws IOException { + Configuration conf = new Configuration(UTIL.getConfiguration()); + conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, FileBasedStoreFileTracker.class, + StoreFileTracker.class); + region = + HBaseTestingUtil.createRegionAndWAL(RI, UTIL.getDataTestDir(name.getMethodName()), conf, TD); + } + + @After + public void tearDown() throws IOException { + if (region != null) { + HBaseTestingUtil.closeRegionAndWAL(region); + } + UTIL.cleanupTestDir(); + } + + @Test + public void testFlushAndCompaction() throws IOException { + for (int i = 0; i < 10; i++) { + for (int j = 0; j < 10; j++) { + int v = i * 10 + j; + region.put(new Put(Bytes.toBytes(v)).addColumn(CF, CQ, Bytes.toBytes(v))); + } + region.flush(true); + if (i % 3 == 2) { + region.compact(true); + } + } + // reopen the region, make sure the store file tracker works, i.e, we can get all the records + // back + region.close(); + region = HRegion.openHRegion(region, null); + for (int i = 0; i < 100; i++) { + Result result = region.get(new Get(Bytes.toBytes(i))); + assertEquals(i, Bytes.toInt(result.getValue(CF, CQ))); + } + } +} From 88321fcc1a0c3709f6f257014a1727f9a69db02d Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Wed, 8 Sep 2021 10:31:49 +0100 Subject: [PATCH 03/26] HBASE-26079 Use StoreFileTracker when splitting and merging (#3617) Signed-off-by: Duo Zhang --- .../MergeTableRegionsProcedure.java | 22 +- .../assignment/SplitTableRegionProcedure.java | 42 +-- .../hbase/regionserver/HRegionFileSystem.java | 42 ++- .../DefaultStoreFileTracker.java | 4 +- .../storefiletracker/StoreFileTracker.java | 1 - .../StoreFileTrackerFactory.java | 33 ++- .../regionserver/TestDefaultStoreEngine.java | 1 + .../TestDirectStoreSplitsMerges.java | 32 ++- .../hbase/regionserver/TestHStoreFile.java | 19 +- .../TestMergesSplitsAddToTracker.java | 262 ++++++++++++++++++ .../regionserver/TestStripeStoreEngine.java | 1 + .../TestStoreFileTracker.java | 56 ++++ 12 files changed, 471 insertions(+), 44 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index da3d73ea852d..e6bbe445d077 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -24,7 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.stream.Stream; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -56,6 +55,8 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreUtils; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.wal.WALSplitUtil; @@ -587,30 +588,35 @@ private void createMergedRegion(final MasterProcedureEnv env) throws IOException final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable()); final FileSystem fs = mfs.getFileSystem(); - + List mergedFiles = new ArrayList<>(); HRegionFileSystem mergeRegionFs = HRegionFileSystem.createRegionOnFileSystem( env.getMasterConfiguration(), fs, tableDir, mergedRegion); for (RegionInfo ri: this.regionsToMerge) { HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( env.getMasterConfiguration(), fs, tableDir, ri, false); - mergeStoreFiles(env, regionFs, mergeRegionFs, mergedRegion); + mergedFiles.addAll(mergeStoreFiles(env, regionFs, mergeRegionFs, mergedRegion)); } assert mergeRegionFs != null; - mergeRegionFs.commitMergedRegion(); + mergeRegionFs.commitMergedRegion(mergedFiles, env); // Prepare to create merged regions env.getAssignmentManager().getRegionStates(). getOrCreateRegionStateNode(mergedRegion).setState(State.MERGING_NEW); } - private void mergeStoreFiles(MasterProcedureEnv env, HRegionFileSystem regionFs, + private List mergeStoreFiles(MasterProcedureEnv env, HRegionFileSystem regionFs, HRegionFileSystem mergeRegionFs, RegionInfo mergedRegion) throws IOException { final TableDescriptor htd = env.getMasterServices().getTableDescriptors() .get(mergedRegion.getTable()); + List mergedFiles = new ArrayList<>(); for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { String family = hcd.getNameAsString(); - final Collection storeFiles = regionFs.getStoreFiles(family); + Configuration trackerConfig = + StoreFileTrackerFactory.mergeConfigurations(env.getMasterConfiguration(), htd, hcd); + StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, true, + family, regionFs); + final Collection storeFiles = tracker.load(); if (storeFiles != null && storeFiles.size() > 0) { final Configuration storeConfiguration = StoreUtils.createStoreConfiguration(env.getMasterConfiguration(), htd, hcd); @@ -622,11 +628,13 @@ private void mergeStoreFiles(MasterProcedureEnv env, HRegionFileSystem regionFs, // is running in a regionserver's Store context, or we might not be able // to read the hfiles. storeFileInfo.setConf(storeConfiguration); - mergeRegionFs.mergeStoreFile(regionFs.getRegionInfo(), family, + Path refFile = mergeRegionFs.mergeStoreFile(regionFs.getRegionInfo(), family, new HStoreFile(storeFileInfo, hcd.getBloomFilterType(), CacheConfig.DISABLED)); + mergedFiles.add(refFile); } } } + return mergedFiles; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index fbd87290d8c2..ff16dc5514b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -33,7 +33,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -66,6 +65,8 @@ import org.apache.hadoop.hbase.regionserver.RegionSplitRestriction; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreUtils; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -621,21 +622,20 @@ public void createDaughterRegions(final MasterProcedureEnv env) throws IOExcepti final FileSystem fs = mfs.getFileSystem(); HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( env.getMasterConfiguration(), fs, tabledir, getParentRegion(), false); - regionFs.createSplitsDir(daughterOneRI, daughterTwoRI); - Pair expectedReferences = splitStoreFiles(env, regionFs); + Pair, List> expectedReferences = splitStoreFiles(env, regionFs); - assertSplitResultFilesCount(fs, expectedReferences.getFirst(), + assertSplitResultFilesCount(fs, expectedReferences.getFirst().size(), regionFs.getSplitsDir(daughterOneRI)); - regionFs.commitDaughterRegion(daughterOneRI); - assertSplitResultFilesCount(fs, expectedReferences.getFirst(), + regionFs.commitDaughterRegion(daughterOneRI, expectedReferences.getFirst(), env); + assertSplitResultFilesCount(fs, expectedReferences.getFirst().size(), new Path(tabledir, daughterOneRI.getEncodedName())); - assertSplitResultFilesCount(fs, expectedReferences.getSecond(), + assertSplitResultFilesCount(fs, expectedReferences.getSecond().size(), regionFs.getSplitsDir(daughterTwoRI)); - regionFs.commitDaughterRegion(daughterTwoRI); - assertSplitResultFilesCount(fs, expectedReferences.getSecond(), + regionFs.commitDaughterRegion(daughterTwoRI, expectedReferences.getSecond(), env); + assertSplitResultFilesCount(fs, expectedReferences.getSecond().size(), new Path(tabledir, daughterTwoRI.getEncodedName())); } @@ -652,7 +652,7 @@ private void deleteDaughterRegions(final MasterProcedureEnv env) throws IOExcept * Create Split directory * @param env MasterProcedureEnv */ - private Pair splitStoreFiles(final MasterProcedureEnv env, + private Pair, List> splitStoreFiles(final MasterProcedureEnv env, final HRegionFileSystem regionFs) throws IOException { final Configuration conf = env.getMasterConfiguration(); TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName()); @@ -668,7 +668,11 @@ private Pair splitStoreFiles(final MasterProcedureEnv env, new HashMap>(htd.getColumnFamilyCount()); for (ColumnFamilyDescriptor cfd : htd.getColumnFamilies()) { String family = cfd.getNameAsString(); - Collection sfis = regionFs.getStoreFiles(family); + Configuration trackerConfig = StoreFileTrackerFactory. + mergeConfigurations(env.getMasterConfiguration(), htd, htd.getColumnFamily(cfd.getName())); + StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, true, + family, regionFs); + Collection sfis = tracker.load(); if (sfis == null) { continue; } @@ -694,7 +698,7 @@ private Pair splitStoreFiles(final MasterProcedureEnv env, } if (nbFiles == 0) { // no file needs to be splitted. - return new Pair(0, 0); + return new Pair<>(Collections.emptyList(), Collections.emptyList()); } // Max #threads is the smaller of the number of storefiles or the default max determined above. int maxThreads = Math.min( @@ -752,14 +756,18 @@ private Pair splitStoreFiles(final MasterProcedureEnv env, throw (InterruptedIOException) new InterruptedIOException().initCause(e); } - int daughterA = 0; - int daughterB = 0; + List daughterA = new ArrayList<>(); + List daughterB = new ArrayList<>(); // Look for any exception for (Future> future : futures) { try { Pair p = future.get(); - daughterA += p.getFirst() != null ? 1 : 0; - daughterB += p.getSecond() != null ? 1 : 0; + if(p.getFirst() != null){ + daughterA.add(p.getFirst()); + } + if(p.getSecond() != null){ + daughterB.add(p.getSecond()); + } } catch (InterruptedException e) { throw (InterruptedIOException) new InterruptedIOException().initCause(e); } catch (ExecutionException e) { @@ -772,7 +780,7 @@ private Pair splitStoreFiles(final MasterProcedureEnv env, getParentRegion().getShortNameToLog() + " Daughter A: " + daughterA + " storefiles, Daughter B: " + daughterB + " storefiles."); } - return new Pair(daughterA, daughterB); + return new Pair<>(daughterA, daughterB); } private void assertSplitResultFilesCount(final FileSystem fs, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index 2f5f8d7e34cd..cb30432c38b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -24,7 +24,9 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.UUID; @@ -49,6 +51,9 @@ import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.Reference; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.FSUtils; @@ -595,19 +600,46 @@ void cleanupDaughterRegion(final RegionInfo regionInfo) throws IOException { * @param regionInfo daughter {@link org.apache.hadoop.hbase.client.RegionInfo} * @throws IOException */ - public Path commitDaughterRegion(final RegionInfo regionInfo) - throws IOException { + public Path commitDaughterRegion(final RegionInfo regionInfo, List allRegionFiles, + MasterProcedureEnv env) throws IOException { Path regionDir = this.getSplitsDir(regionInfo); if (fs.exists(regionDir)) { // Write HRI to a file in case we need to recover hbase:meta Path regionInfoFile = new Path(regionDir, REGION_INFO_FILE); byte[] regionInfoContent = getRegionInfoFileContent(regionInfo); writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent); + HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( + env.getMasterConfiguration(), fs, getTableDir(), regionInfo, false); + insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs); } - return regionDir; } + private void insertRegionFilesIntoStoreTracker(List allFiles, MasterProcedureEnv env, + HRegionFileSystem regionFs) throws IOException { + TableDescriptor tblDesc = env.getMasterServices().getTableDescriptors(). + get(regionInfo.getTable()); + //we need to map trackers per store + Map trackerMap = new HashMap<>(); + //we need to map store files per store + Map> fileInfoMap = new HashMap<>(); + for(Path file : allFiles) { + String familyName = file.getParent().getName(); + trackerMap.computeIfAbsent(familyName, t -> { + Configuration config = StoreFileTrackerFactory.mergeConfigurations(conf, tblDesc, + tblDesc.getColumnFamily(Bytes.toBytes(familyName))); + return StoreFileTrackerFactory. + create(config, true, familyName, regionFs); + }); + fileInfoMap.computeIfAbsent(familyName, l -> new ArrayList<>()); + List infos = fileInfoMap.get(familyName); + infos.add(new StoreFileInfo(conf, fs, file, true)); + } + for(Map.Entry entry : trackerMap.entrySet()) { + entry.getValue().add(fileInfoMap.get(entry.getKey())); + } + } + /** * Creates region split daughter directories under the table dir. If the daughter regions already * exist, for example, in the case of a recovery from a previous failed split procedure, this @@ -795,13 +827,15 @@ public Path mergeStoreFile(RegionInfo mergingRegion, String familyName, HStoreFi * Commit a merged region, making it ready for use. * @throws IOException */ - public void commitMergedRegion() throws IOException { + public void commitMergedRegion(List allMergedFiles, MasterProcedureEnv env) + throws IOException { Path regionDir = getMergesDir(regionInfoForFs); if (regionDir != null && fs.exists(regionDir)) { // Write HRI to a file in case we need to recover hbase:meta Path regionInfoFile = new Path(regionDir, REGION_INFO_FILE); byte[] regionInfoContent = getRegionInfoFileContent(regionInfo); writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent); + insertRegionFilesIntoStoreTracker(allMergedFiles, env, this); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java index fa044818336f..22e05132bf91 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; + import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.yetus.audience.InterfaceAudience; @@ -32,8 +33,7 @@ @InterfaceAudience.Private class DefaultStoreFileTracker extends StoreFileTrackerBase { - public DefaultStoreFileTracker(Configuration conf, boolean isPrimaryReplica, - StoreContext ctx) { + public DefaultStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { super(conf, isPrimaryReplica, ctx); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java index aadedc8ef727..0a85abb00a4e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java @@ -48,7 +48,6 @@ */ @InterfaceAudience.Private public interface StoreFileTracker { - /** * Load the store files list when opening a region. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index 6cdfaf4a0fe1..c446d5ae9a31 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -18,22 +18,51 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CompoundConfiguration; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Factory method for creating store file tracker. */ @InterfaceAudience.Private public final class StoreFileTrackerFactory { - public static final String TRACK_IMPL = "hbase.store.file-tracker.impl"; + private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerFactory.class); public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, - StoreContext ctx) { + StoreContext ctx) { Class tracker = conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class); + LOG.info("instantiating StoreFileTracker impl {}", tracker.getName()); return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); } + + public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, String family, + HRegionFileSystem regionFs) { + ColumnFamilyDescriptorBuilder fDescBuilder = + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family)); + StoreContext ctx = StoreContext.getBuilder(). + withColumnFamilyDescriptor(fDescBuilder.build()). + withRegionFileSystem(regionFs). + build(); + return StoreFileTrackerFactory.create(conf, isPrimaryReplica, ctx); + } + + public static Configuration mergeConfigurations(Configuration global, + TableDescriptor table, ColumnFamilyDescriptor family) { + return new CompoundConfiguration() + .add(global) + .addBytesMap(table.getValues()) + .addStringMap(family.getConfiguration()) + .addBytesMap(family.getValues()); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java index 3784876a59f3..523f27782362 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java @@ -67,6 +67,7 @@ public void testCustomParts() throws Exception { DummyStoreFlusher.class.getName()); HRegion mockRegion = Mockito.mock(HRegion.class); HStore mockStore = Mockito.mock(HStore.class); + mockStore.conf = conf; Mockito.when(mockStore.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO); Mockito.when(mockStore.getHRegion()).thenReturn(mockRegion); StoreEngine se = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDirectStoreSplitsMerges.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDirectStoreSplitsMerges.java index bd24f1b22293..0eba8aa541ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDirectStoreSplitsMerges.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDirectStoreSplitsMerges.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import org.apache.hadoop.fs.Path; @@ -32,6 +33,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -139,7 +141,9 @@ public void testCommitDaughterRegionNoFiles() throws Exception { setRegionId(region.getRegionInfo().getRegionId() + EnvironmentEdgeManager.currentTime()).build(); Path splitDir = regionFS.getSplitsDir(daughterA); - Path result = regionFS.commitDaughterRegion(daughterA); + MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster(). + getMasterProcedureExecutor().getEnvironment(); + Path result = regionFS.commitDaughterRegion(daughterA, new ArrayList<>(), env); assertEquals(splitDir, result); } @@ -162,14 +166,18 @@ public void testCommitDaughterRegionWithFiles() throws Exception { Path splitDirA = regionFS.getSplitsDir(daughterA); Path splitDirB = regionFS.getSplitsDir(daughterB); HStoreFile file = (HStoreFile) region.getStore(FAMILY_NAME).getStorefiles().toArray()[0]; - regionFS + List filesA = new ArrayList<>(); + filesA.add(regionFS .splitStoreFile(daughterA, Bytes.toString(FAMILY_NAME), file, - Bytes.toBytes("002"), false, region.getSplitPolicy()); - regionFS + Bytes.toBytes("002"), false, region.getSplitPolicy())); + List filesB = new ArrayList<>(); + filesB.add(regionFS .splitStoreFile(daughterB, Bytes.toString(FAMILY_NAME), file, - Bytes.toBytes("002"), true, region.getSplitPolicy()); - Path resultA = regionFS.commitDaughterRegion(daughterA); - Path resultB = regionFS.commitDaughterRegion(daughterB); + Bytes.toBytes("002"), true, region.getSplitPolicy())); + MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster(). + getMasterProcedureExecutor().getEnvironment(); + Path resultA = regionFS.commitDaughterRegion(daughterA, filesA, env); + Path resultB = regionFS.commitDaughterRegion(daughterB, filesB, env); assertEquals(splitDirA, resultA); assertEquals(splitDirB, resultB); } @@ -203,8 +211,11 @@ public void testCommitMergedRegion() throws Exception { mergeFileFromRegion(mergeRegionFs, first, file); //merge file from second region file = (HStoreFile) second.getStore(FAMILY_NAME).getStorefiles().toArray()[0]; - mergeFileFromRegion(mergeRegionFs, second, file); - mergeRegionFs.commitMergedRegion(); + List mergedFiles = new ArrayList<>(); + mergedFiles.add(mergeFileFromRegion(mergeRegionFs, second, file)); + MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster(). + getMasterProcedureExecutor().getEnvironment(); + mergeRegionFs.commitMergedRegion(mergedFiles, env); } private void waitForSplitProcComplete(int attempts, int waitTime) throws Exception { @@ -223,11 +234,12 @@ private void waitForSplitProcComplete(int attempts, int waitTime) throws Excepti } } - private void mergeFileFromRegion(HRegionFileSystem regionFS, HRegion regionToMerge, + private Path mergeFileFromRegion(HRegionFileSystem regionFS, HRegion regionToMerge, HStoreFile file) throws IOException { Path mergedFile = regionFS.mergeStoreFile(regionToMerge.getRegionInfo(), Bytes.toString(FAMILY_NAME), file); validateResultingFile(regionToMerge.getRegionInfo().getEncodedName(), mergedFile); + return mergedFile; } private void validateResultingFile(String originalRegion, Path result){ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java index cdef341965b8..394e62d556f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -49,12 +50,14 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; @@ -69,6 +72,8 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.io.hfile.ReaderContext; import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder; +import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.BloomFilterFactory; @@ -1060,7 +1065,19 @@ private Path splitStoreFile(final HRegionFileSystem regionFs, final RegionInfo h if (null == path) { return null; } - Path regionDir = regionFs.commitDaughterRegion(hri); + List splitFiles = new ArrayList<>(); + splitFiles.add(path); + MasterProcedureEnv mockEnv = mock(MasterProcedureEnv.class); + MasterServices mockServices = mock(MasterServices.class); + when(mockEnv.getMasterServices()).thenReturn(mockServices); + when(mockEnv.getMasterConfiguration()).thenReturn(new Configuration()); + TableDescriptors mockTblDescs = mock(TableDescriptors.class); + when(mockServices.getTableDescriptors()).thenReturn(mockTblDescs); + TableDescriptor mockTblDesc = mock(TableDescriptor.class); + when(mockTblDescs.get(any())).thenReturn(mockTblDesc); + ColumnFamilyDescriptor mockCfDesc = mock(ColumnFamilyDescriptor.class); + when(mockTblDesc.getColumnFamily(any())).thenReturn(mockCfDesc); + Path regionDir = regionFs.commitDaughterRegion(hri, splitFiles, mockEnv); return new Path(new Path(regionDir, family), path.getName()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java new file mode 100644 index 000000000000..c6205cb18493 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java @@ -0,0 +1,262 @@ +/** + * 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 static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory. + TRACK_IMPL; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.lang3.mutable.MutableBoolean; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.regionserver.storefiletracker.TestStoreFileTracker; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + + +@Category({RegionServerTests.class, LargeTests.class}) +public class TestMergesSplitsAddToTracker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMergesSplitsAddToTracker.class); + + private static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + + public static final byte[] FAMILY_NAME = Bytes.toBytes("info"); + + @Rule + public TestName name = new TestName(); + + @BeforeClass + public static void setupClass() throws Exception { + TEST_UTIL.getConfiguration().set(TRACK_IMPL, TestStoreFileTracker.class.getName()); + TEST_UTIL.startMiniCluster(); + } + + @AfterClass + public static void afterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Before + public void setup(){ + TestStoreFileTracker.trackedFiles = new HashMap<>(); + } + + @Test + public void testCommitDaughterRegion() throws Exception { + TableName table = TableName.valueOf(name.getMethodName()); + TEST_UTIL.createTable(table, FAMILY_NAME); + //first put some data in order to have a store file created + putThreeRowsAndFlush(table); + HRegion region = TEST_UTIL.getHBaseCluster().getRegions(table).get(0); + HRegionFileSystem regionFS = region.getStores().get(0).getRegionFileSystem(); + RegionInfo daughterA = + RegionInfoBuilder.newBuilder(table).setStartKey(region.getRegionInfo().getStartKey()). + setEndKey(Bytes.toBytes("002")).setSplit(false). + setRegionId(region.getRegionInfo().getRegionId() + + EnvironmentEdgeManager.currentTime()). + build(); + RegionInfo daughterB = RegionInfoBuilder.newBuilder(table).setStartKey(Bytes.toBytes("002")) + .setEndKey(region.getRegionInfo().getEndKey()).setSplit(false) + .setRegionId(region.getRegionInfo().getRegionId()).build(); + HStoreFile file = (HStoreFile) region.getStore(FAMILY_NAME).getStorefiles().toArray()[0]; + List splitFilesA = new ArrayList<>(); + splitFilesA.add(regionFS + .splitStoreFile(daughterA, Bytes.toString(FAMILY_NAME), file, + Bytes.toBytes("002"), false, region.getSplitPolicy())); + List splitFilesB = new ArrayList<>(); + splitFilesB.add(regionFS + .splitStoreFile(daughterB, Bytes.toString(FAMILY_NAME), file, + Bytes.toBytes("002"), true, region.getSplitPolicy())); + MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster(). + getMasterProcedureExecutor().getEnvironment(); + Path resultA = regionFS.commitDaughterRegion(daughterA, splitFilesA, env); + Path resultB = regionFS.commitDaughterRegion(daughterB, splitFilesB, env); + FileSystem fs = regionFS.getFileSystem(); + verifyFilesAreTracked(resultA, fs); + verifyFilesAreTracked(resultB, fs); + } + + @Test + public void testCommitMergedRegion() throws Exception { + TableName table = TableName.valueOf(name.getMethodName()); + TEST_UTIL.createTable(table, FAMILY_NAME); + //splitting the table first + TEST_UTIL.getAdmin().split(table, Bytes.toBytes("002")); + //Add data and flush to create files in the two different regions + putThreeRowsAndFlush(table); + List regions = TEST_UTIL.getHBaseCluster().getRegions(table); + HRegion first = regions.get(0); + HRegion second = regions.get(1); + HRegionFileSystem regionFS = first.getRegionFileSystem(); + + RegionInfo mergeResult = + RegionInfoBuilder.newBuilder(table).setStartKey(first.getRegionInfo().getStartKey()) + .setEndKey(second.getRegionInfo().getEndKey()).setSplit(false) + .setRegionId(first.getRegionInfo().getRegionId() + + EnvironmentEdgeManager.currentTime()).build(); + + HRegionFileSystem mergeFS = HRegionFileSystem.createRegionOnFileSystem( + TEST_UTIL.getHBaseCluster().getMaster().getConfiguration(), + regionFS.getFileSystem(), regionFS.getTableDir(), mergeResult); + + List mergedFiles = new ArrayList<>(); + //merge file from first region + mergedFiles.add(mergeFileFromRegion(first, mergeFS)); + //merge file from second region + mergedFiles.add(mergeFileFromRegion(second, mergeFS)); + MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster(). + getMasterProcedureExecutor().getEnvironment(); + mergeFS.commitMergedRegion(mergedFiles, env); + //validate + FileSystem fs = first.getRegionFileSystem().getFileSystem(); + Path finalMergeDir = new Path(first.getRegionFileSystem().getTableDir(), + mergeResult.getEncodedName()); + verifyFilesAreTracked(finalMergeDir, fs); + } + + @Test + public void testSplitLoadsFromTracker() throws Exception { + TableName table = TableName.valueOf(name.getMethodName()); + TEST_UTIL.createTable(table, FAMILY_NAME); + //Add data and flush to create files in the two different regions + putThreeRowsAndFlush(table); + HRegion region = TEST_UTIL.getHBaseCluster().getRegions(table).get(0); + Pair copyResult = copyFileInTheStoreDir(region); + StoreFileInfo fileInfo = copyResult.getFirst(); + String copyName = copyResult.getSecond(); + //Now splits the region + TEST_UTIL.getAdmin().split(table, Bytes.toBytes("002")); + List regions = TEST_UTIL.getHBaseCluster().getRegions(table); + HRegion first = regions.get(0); + validateDaughterRegionsFiles(first, fileInfo.getActiveFileName(), copyName); + HRegion second = regions.get(1); + validateDaughterRegionsFiles(second, fileInfo.getActiveFileName(), copyName); + } + + @Test + public void testMergeLoadsFromTracker() throws Exception { + TableName table = TableName.valueOf(name.getMethodName()); + TEST_UTIL.createTable(table, new byte[][]{FAMILY_NAME}, + new byte[][]{Bytes.toBytes("002")}); + //Add data and flush to create files in the two different regions + putThreeRowsAndFlush(table); + List regions = TEST_UTIL.getHBaseCluster().getRegions(table); + HRegion first = regions.get(0); + Pair copyResult = copyFileInTheStoreDir(first); + StoreFileInfo fileInfo = copyResult.getFirst(); + String copyName = copyResult.getSecond(); + //Now merges the first two regions + TEST_UTIL.getAdmin().mergeRegionsAsync(new byte[][]{ + first.getRegionInfo().getEncodedNameAsBytes(), + regions.get(1).getRegionInfo().getEncodedNameAsBytes() + }, true).get(10, TimeUnit.SECONDS); + regions = TEST_UTIL.getHBaseCluster().getRegions(table); + HRegion merged = regions.get(0); + validateDaughterRegionsFiles(merged, fileInfo.getActiveFileName(), copyName); + } + + private Pair copyFileInTheStoreDir(HRegion region) throws IOException { + Path storeDir = region.getRegionFileSystem().getStoreDir("info"); + //gets the single file + StoreFileInfo fileInfo = region.getRegionFileSystem().getStoreFiles("info").get(0); + //make a copy of the valid file staight into the store dir, so that it's not tracked. + String copyName = UUID.randomUUID().toString().replaceAll("-", ""); + Path copy = new Path(storeDir, copyName); + FileUtil.copy(region.getFilesystem(), fileInfo.getFileStatus(), region.getFilesystem(), + copy , false, false, TEST_UTIL.getConfiguration()); + return new Pair<>(fileInfo, copyName); + } + + private void validateDaughterRegionsFiles(HRegion region, String orignalFileName, + String untrackedFile) throws IOException { + //verify there's no link for the untracked, copied file in first region + List infos = region.getRegionFileSystem().getStoreFiles("info"); + final MutableBoolean foundLink = new MutableBoolean(false); + infos.stream().forEach(i -> { + i.getActiveFileName().contains(orignalFileName); + if(i.getActiveFileName().contains(untrackedFile)){ + fail(); + } + if(i.getActiveFileName().contains(orignalFileName)){ + foundLink.setTrue(); + } + }); + assertTrue(foundLink.booleanValue()); + } + + private void verifyFilesAreTracked(Path regionDir, FileSystem fs) throws Exception { + String storeId = regionDir.getName() + "-info"; + for(FileStatus f : fs.listStatus(new Path(regionDir, Bytes.toString(FAMILY_NAME)))){ + assertTrue(TestStoreFileTracker.trackedFiles.get(storeId).stream().filter(s -> + s.getPath().equals(f.getPath())).findFirst().isPresent()); + } + } + + private Path mergeFileFromRegion(HRegion regionToMerge, HRegionFileSystem mergeFS) + throws IOException { + HStoreFile file = (HStoreFile) regionToMerge.getStore(FAMILY_NAME).getStorefiles().toArray()[0]; + return mergeFS.mergeStoreFile(regionToMerge.getRegionInfo(), Bytes.toString(FAMILY_NAME), file); + } + + private void putThreeRowsAndFlush(TableName table) throws IOException { + Table tbl = TEST_UTIL.getConnection().getTable(table); + Put put = new Put(Bytes.toBytes("001")); + byte[] qualifier = Bytes.toBytes("1"); + put.addColumn(FAMILY_NAME, qualifier, Bytes.toBytes(1)); + tbl.put(put); + put = new Put(Bytes.toBytes("002")); + put.addColumn(FAMILY_NAME, qualifier, Bytes.toBytes(2)); + tbl.put(put); + put = new Put(Bytes.toBytes("003")); + put.addColumn(FAMILY_NAME, qualifier, Bytes.toBytes(2)); + tbl.put(put); + TEST_UTIL.flush(table); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java index eb0b1c1ca694..80012dfcd461 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java @@ -120,6 +120,7 @@ private static HStoreFile createFile() throws Exception { private static TestStoreEngine createEngine(Configuration conf) throws Exception { HRegion region = mock(HRegion.class); HStore store = mock(HStore.class); + store.conf = conf; when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO); when(store.getHRegion()).thenReturn(region); CellComparatorImpl kvComparator = mock(CellComparatorImpl.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java new file mode 100644 index 000000000000..05ca1fcb419b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java @@ -0,0 +1,56 @@ +/** + * 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.storefiletracker; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestStoreFileTracker extends DefaultStoreFileTracker { + + private static final Logger LOG = LoggerFactory.getLogger(TestStoreFileTracker.class); + public static Map> trackedFiles = new HashMap<>(); + private String storeId; + + public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { + super(conf, isPrimaryReplica, ctx); + this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString(); + LOG.info("created storeId: {}", storeId); + trackedFiles.computeIfAbsent(storeId, v -> new ArrayList<>()); + } + + @Override + protected void doAddNewStoreFiles(Collection newFiles) throws IOException { + LOG.info("adding to storeId: {}", storeId); + trackedFiles.get(storeId).addAll(newFiles); + } + + @Override + public List load() throws IOException { + return trackedFiles.get(storeId); + } +} From 0ae94b8b4ceeca26a3d3bbbce845665c4b35294f Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 9 Sep 2021 12:11:25 +0800 Subject: [PATCH 04/26] HBASE-26224 Introduce a MigrationStoreFileTracker to support migrating from different store file tracker implementations (#3656) Signed-off-by: Wellington Chevreuil --- .../DefaultStoreFileTracker.java | 10 +- .../FileBasedStoreFileTracker.java | 15 +- .../MigrationStoreFileTracker.java | 88 ++++++++ .../storefiletracker/StoreFileListFile.java | 6 +- .../StoreFileTrackerBase.java | 12 +- .../StoreFileTrackerFactory.java | 40 ++-- .../TestMigrationStoreFileTracker.java | 193 ++++++++++++++++++ 7 files changed, 343 insertions(+), 21 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java index 22e05132bf91..a13b75b0dcc4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.Collections; import java.util.List; import org.apache.hadoop.conf.Configuration; @@ -39,7 +40,9 @@ public DefaultStoreFileTracker(Configuration conf, boolean isPrimaryReplica, Sto @Override public List load() throws IOException { - return ctx.getRegionFileSystem().getStoreFiles(ctx.getFamily().getNameAsString()); + List files = + ctx.getRegionFileSystem().getStoreFiles(ctx.getFamily().getNameAsString()); + return files != null ? files : Collections.emptyList(); } @Override @@ -57,4 +60,9 @@ protected void doAddCompactionResults(Collection compactedFiles, Collection newFiles) throws IOException { // NOOP } + + @Override + void set(List files) { + // NOOP + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java index de28b0eb9996..c370b87c1154 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java @@ -48,7 +48,7 @@ * storages. */ @InterfaceAudience.Private -public class FileBasedStoreFileTracker extends StoreFileTrackerBase { +class FileBasedStoreFileTracker extends StoreFileTrackerBase { private final StoreFileListFile backedFile; @@ -139,4 +139,17 @@ protected void doAddCompactionResults(Collection compactedFiles, } } } + + @Override + void set(List files) throws IOException { + synchronized (storefiles) { + storefiles.clear(); + StoreFileList.Builder builder = StoreFileList.newBuilder(); + for (StoreFileInfo info : files) { + storefiles.put(info.getPath().getName(), info); + builder.addStoreFile(toStoreFileEntry(info)); + } + backedFile.update(builder); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java new file mode 100644 index 000000000000..e486e6d563af --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java @@ -0,0 +1,88 @@ +/** + * 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.storefiletracker; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + +/** + * A store file tracker used for migrating between store file tracker implementations. + */ +@InterfaceAudience.Private +class MigrationStoreFileTracker extends StoreFileTrackerBase { + + public static final String SRC_IMPL = "hbase.store.file-tracker.migration.src.impl"; + + public static final String DST_IMPL = "hbase.store.file-tracker.migration.dst.impl"; + + private final StoreFileTrackerBase src; + + private final StoreFileTrackerBase dst; + + public MigrationStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { + super(conf, isPrimaryReplica, ctx); + this.src = StoreFileTrackerFactory.create(conf, SRC_IMPL, isPrimaryReplica, ctx); + this.dst = StoreFileTrackerFactory.create(conf, DST_IMPL, isPrimaryReplica, ctx); + Preconditions.checkArgument(!src.getClass().equals(dst.getClass()), + "src and dst is the same: %s", src.getClass()); + } + + @Override + public List load() throws IOException { + List files = src.load(); + dst.set(files); + return files; + } + + @Override + protected boolean requireWritingToTmpDirFirst() { + // Returns true if either of the two StoreFileTracker returns true. + // For example, if we want to migrate from a tracker implementation which can ignore the broken + // files under data directory to a tracker implementation which can not, if we still allow + // writing in tmp directory directly, we may have some broken files under the data directory and + // then after we finally change the implementation which can not ignore the broken files, we + // will be in trouble. + return src.requireWritingToTmpDirFirst() || dst.requireWritingToTmpDirFirst(); + } + + @Override + protected void doAddNewStoreFiles(Collection newFiles) throws IOException { + src.doAddNewStoreFiles(newFiles); + dst.doAddNewStoreFiles(newFiles); + } + + @Override + protected void doAddCompactionResults(Collection compactedFiles, + Collection newFiles) throws IOException { + src.doAddCompactionResults(compactedFiles, newFiles); + dst.doAddCompactionResults(compactedFiles, newFiles); + } + + @Override + void set(List files) { + throw new UnsupportedOperationException( + "Should not call this method on " + getClass().getSimpleName()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java index c778bfc51deb..ffb3647e6259 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java @@ -29,7 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; @@ -121,7 +120,10 @@ StoreFileList load() throws IOException { * We will set the timestamp in this method so just pass the builder in */ void update(StoreFileList.Builder builder) throws IOException { - Preconditions.checkState(nextTrackFile >= 0, "should call load first before calling update"); + if (nextTrackFile < 0) { + // we need to call load first to load the prevTimestamp and also the next file + load(); + } FileSystem fs = ctx.getRegionFileSystem().getFileSystem(); long timestamp = Math.max(prevTimestamp + 1, EnvironmentEdgeManager.currentTime()); try (FSDataOutputStream out = fs.create(trackFiles[nextTrackFile], true)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java index 92c699278c2e..d860f8e3812d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -95,8 +96,7 @@ private HFileContext createFileContext(Compression.Algorithm compression, } @Override - public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) - throws IOException { + public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException { if (!isPrimaryReplica) { throw new IllegalStateException("Should not call create writer on secondary replicas"); } @@ -170,4 +170,12 @@ public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) protected abstract void doAddCompactionResults(Collection compactedFiles, Collection newFiles) throws IOException; + + /** + * used to mirror the store file list after loading when migration. + *

+ * Do not add this method to the {@link StoreFileTracker} interface since we do not need this + * method in upper layer. + */ + abstract void set(List files) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index c446d5ae9a31..6eb269ee2b40 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -18,18 +18,20 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CompoundConfiguration; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + /** * Factory method for creating store file tracker. */ @@ -39,7 +41,7 @@ public final class StoreFileTrackerFactory { private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerFactory.class); public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, - StoreContext ctx) { + StoreContext ctx) { Class tracker = conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class); LOG.info("instantiating StoreFileTracker impl {}", tracker.getName()); @@ -47,22 +49,30 @@ public static StoreFileTracker create(Configuration conf, boolean isPrimaryRepli } public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, String family, - HRegionFileSystem regionFs) { + HRegionFileSystem regionFs) { ColumnFamilyDescriptorBuilder fDescBuilder = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family)); - StoreContext ctx = StoreContext.getBuilder(). - withColumnFamilyDescriptor(fDescBuilder.build()). - withRegionFileSystem(regionFs). - build(); - return StoreFileTrackerFactory.create(conf, isPrimaryReplica, ctx); + StoreContext ctx = StoreContext.getBuilder().withColumnFamilyDescriptor(fDescBuilder.build()) + .withRegionFileSystem(regionFs).build(); + return StoreFileTrackerFactory.create(conf, TRACK_IMPL, isPrimaryReplica, ctx); + } + + public static Configuration mergeConfigurations(Configuration global, TableDescriptor table, + ColumnFamilyDescriptor family) { + return StoreUtils.createStoreConfiguration(global, table, family); } - public static Configuration mergeConfigurations(Configuration global, - TableDescriptor table, ColumnFamilyDescriptor family) { - return new CompoundConfiguration() - .add(global) - .addBytesMap(table.getValues()) - .addStringMap(family.getConfiguration()) - .addBytesMap(family.getValues()); + static StoreFileTrackerBase create(Configuration conf, String configName, + boolean isPrimaryReplica, StoreContext ctx) { + String className = + Preconditions.checkNotNull(conf.get(configName), "config %s is not set", configName); + Class tracker; + try { + tracker = Class.forName(className).asSubclass(StoreFileTrackerBase.class); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + LOG.info("instantiating StoreFileTracker impl {} as {}", tracker.getName(), configName); + return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java new file mode 100644 index 000000000000..e9ebb4c792e4 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java @@ -0,0 +1,193 @@ +/** + * 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.storefiletracker; + +import static org.hamcrest.CoreMatchers.hasItems; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.regionserver.ChunkCreator; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.MemStoreLAB; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; + +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; +import org.apache.hbase.thirdparty.com.google.common.io.Closeables; + +@RunWith(Parameterized.class) +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestMigrationStoreFileTracker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMigrationStoreFileTracker.class); + + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + + private static final byte[] CF = Bytes.toBytes("cf"); + + private static final byte[] CQ = Bytes.toBytes("cq"); + + private static final TableDescriptor TD = + TableDescriptorBuilder.newBuilder(TableName.valueOf("file_based_tracker")) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build(); + + private static final RegionInfo RI = RegionInfoBuilder.newBuilder(TD.getTableName()).build(); + + @Rule + public TestName name = new TestName(); + + @Parameter(0) + public Class srcImplClass; + + @Parameter(1) + public Class dstImplClass; + + private HRegion region; + + private Path rootDir; + + private WAL wal; + + @Parameters(name = "{index}: src={0}, dst={1}") + public static List params() { + List> impls = + Arrays.asList(DefaultStoreFileTracker.class, FileBasedStoreFileTracker.class); + List params = new ArrayList<>(); + for (Class src : impls) { + for (Class dst : impls) { + if (src.equals(dst)) { + continue; + } + params.add(new Object[] { src, dst }); + } + } + return params; + } + + @BeforeClass + public static void setUpBeforeClass() { + ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null, + MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT); + } + + @Before + public void setUp() throws IOException { + Configuration conf = UTIL.getConfiguration(); + conf.setClass(MigrationStoreFileTracker.SRC_IMPL, srcImplClass, StoreFileTrackerBase.class); + conf.setClass(MigrationStoreFileTracker.DST_IMPL, dstImplClass, StoreFileTrackerBase.class); + rootDir = UTIL.getDataTestDir(name.getMethodName().replaceAll("[=:\\[ ]", "_")); + wal = HBaseTestingUtil.createWal(conf, rootDir, RI); + } + + @After + public void tearDown() throws IOException { + if (region != null) { + region.close(); + } + Closeables.close(wal, true); + UTIL.cleanupTestDir(); + } + + private List getStoreFiles() { + return Iterables.getOnlyElement(region.getStores()).getStorefiles().stream() + .map(s -> s.getFileInfo().getPath().getName()).collect(Collectors.toList()); + } + + private HRegion createRegion(Class trackerImplClass) + throws IOException { + Configuration conf = new Configuration(UTIL.getConfiguration()); + conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, trackerImplClass, StoreFileTracker.class); + return HRegion.createHRegion(RI, rootDir, conf, TD, wal, true); + } + + private void reopenRegion(Class trackerImplClass) + throws IOException { + region.flush(true); + List before = getStoreFiles(); + region.close(); + Configuration conf = new Configuration(UTIL.getConfiguration()); + conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, trackerImplClass, StoreFileTracker.class); + region = HRegion.openHRegion(rootDir, RI, TD, wal, conf); + List after = getStoreFiles(); + assertEquals(before.size(), after.size()); + assertThat(after, hasItems(before.toArray(new String[0]))); + } + + private void putData(int start, int end) throws IOException { + for (int i = start; i < end; i++) { + region.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); + if (i % 30 == 0) { + region.flush(true); + } + } + } + + private void verifyData(int start, int end) throws IOException { + for (int i = start; i < end; i++) { + Result result = region.get(new Get(Bytes.toBytes(i))); + assertEquals(i, Bytes.toInt(result.getValue(CF, CQ))); + } + } + + @Test + public void testMigration() throws IOException { + region = createRegion(srcImplClass); + putData(0, 100); + verifyData(0, 100); + reopenRegion(MigrationStoreFileTracker.class); + verifyData(0, 100); + region.compact(true); + putData(100, 200); + reopenRegion(dstImplClass); + verifyData(0, 200); + } +} From c184a9261e3fac790ac6d4889d9d404f862e3932 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Sun, 12 Sep 2021 14:14:03 +0100 Subject: [PATCH 05/26] HBASE-26246 Persist the StoreFileTracker configurations to TableDescriptor when creating table (#3666) Signed-off-by: Duo Zhang --- .../procedure/CreateTableProcedure.java | 7 ++++++ .../hbase/regionserver/HRegionFileSystem.java | 2 +- .../MigrationStoreFileTracker.java | 8 ++++++ .../storefiletracker/StoreFileTracker.java | 8 ++++++ .../StoreFileTrackerBase.java | 13 ++++++++++ .../StoreFileTrackerFactory.java | 25 +++++++++++++------ .../apache/hadoop/hbase/client/TestAdmin.java | 6 +++++ .../hadoop/hbase/client/TestAdmin3.java | 6 +++++ .../hbase/client/TestAsyncTableAdminApi.java | 6 +++++ .../hbase/client/TestAsyncTableAdminApi3.java | 6 +++++ .../MasterProcedureTestingUtility.java | 7 ++++++ .../procedure/TestCreateTableProcedure.java | 17 +++++++++++++ .../TestStoreFileTracker.java | 14 ++++++++--- 13 files changed, 113 insertions(+), 12 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java index 2313e70f75bb..dccea5512c1f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.procedure; + import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -33,10 +34,12 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.rsgroup.RSGroupInfo; import org.apache.hadoop.hbase.util.CommonFSUtils; @@ -285,6 +288,10 @@ private void preCreate(final MasterProcedureEnv env) getTableName(), (newRegions != null ? newRegions.size() : 0)); } + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); + StoreFileTrackerFactory.persistTrackerConfig(env.getMasterConfiguration(), builder); + tableDescriptor = builder.build(); + final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); if (cpHost != null) { final RegionInfo[] regions = newRegions == null ? null : diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index cb30432c38b1..aa0ee273ea9e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -610,7 +610,7 @@ public Path commitDaughterRegion(final RegionInfo regionInfo, List allRegi writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent); HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( env.getMasterConfiguration(), fs, getTableDir(), regionInfo, false); - insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs); + insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs); } return regionDir; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java index e486e6d563af..483a240baded 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.yetus.audience.InterfaceAudience; @@ -85,4 +86,11 @@ void set(List files) { throw new UnsupportedOperationException( "Should not call this method on " + getClass().getSimpleName()); } + + @Override + public void persistConfiguration(TableDescriptorBuilder builder) { + super.persistConfiguration(builder); + builder.setValue(SRC_IMPL, src.getClass().getName()); + builder.setValue(DST_IMPL, dst.getClass().getName()); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java index 0a85abb00a4e..81fa1a9be5b2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.util.Collection; import java.util.List; + +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; @@ -71,4 +73,10 @@ void replace(Collection compactedFiles, Collection * @return Writer for a new StoreFile */ StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException; + + /** + * Saves StoreFileTracker implementations specific configs into the table descriptors. + * @param builder The table descriptor builder for the given table. + */ + void persistConfiguration(TableDescriptorBuilder builder); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java index d860f8e3812d..83ebbc78ab35 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -17,18 +17,22 @@ */ package org.apache.hadoop.hbase.regionserver.storefiletracker; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; + import java.io.IOException; import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; @@ -78,6 +82,15 @@ public final void replace(Collection compactedFiles, } } + @Override + public void persistConfiguration(TableDescriptorBuilder builder) { + if (StringUtils.isEmpty(builder.getValue(TRACK_IMPL))) { + String trackerImpl = StoreFileTrackerFactory. + getStoreFileTrackerImpl(conf).getName(); + builder.setValue(TRACK_IMPL, trackerImpl).build(); + } + } + private HFileContext createFileContext(Compression.Algorithm compression, boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context encryptionContext) { if (compression == null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index 6eb269ee2b40..b9ec713cf235 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -6,9 +6,7 @@ * 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 - * + * 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. @@ -21,6 +19,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreUtils; @@ -35,15 +34,17 @@ /** * Factory method for creating store file tracker. */ -@InterfaceAudience.Private -public final class StoreFileTrackerFactory { +@InterfaceAudience.Private public final class StoreFileTrackerFactory { public static final String TRACK_IMPL = "hbase.store.file-tracker.impl"; private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerFactory.class); + public static Class getStoreFileTrackerImpl(Configuration conf) { + return conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class); + } + public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { - Class tracker = - conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class); + Class tracker = getStoreFileTrackerImpl(conf); LOG.info("instantiating StoreFileTracker impl {}", tracker.getName()); return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); } @@ -54,7 +55,7 @@ public static StoreFileTracker create(Configuration conf, boolean isPrimaryRepli ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family)); StoreContext ctx = StoreContext.getBuilder().withColumnFamilyDescriptor(fDescBuilder.build()) .withRegionFileSystem(regionFs).build(); - return StoreFileTrackerFactory.create(conf, TRACK_IMPL, isPrimaryReplica, ctx); + return StoreFileTrackerFactory.create(conf, isPrimaryReplica, ctx); } public static Configuration mergeConfigurations(Configuration global, TableDescriptor table, @@ -75,4 +76,12 @@ static StoreFileTrackerBase create(Configuration conf, String configName, LOG.info("instantiating StoreFileTracker impl {} as {}", tracker.getName(), configName); return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); } + + public static void persistTrackerConfig(Configuration conf, TableDescriptorBuilder builder) { + TableDescriptor tableDescriptor = builder.build(); + ColumnFamilyDescriptor cfDesc = tableDescriptor.getColumnFamilies()[0]; + StoreContext context = StoreContext.getBuilder().withColumnFamilyDescriptor(cfDesc).build(); + StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, context); + tracker.persistConfiguration(builder); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 22302629dc70..10ab4961a5b4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.HBaseTestingUtil.countRows; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -423,6 +425,10 @@ private void testCloneTableSchema(final TableName tableName, final TableName new assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize()); assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled()); assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive()); + // HBASE-26246 introduced persist of store file tracker into table descriptor + tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACK_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()). + build(); TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc); if (preserveSplits) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java index e7be08e41edf..7d40fd12bda5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.client; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -236,6 +238,10 @@ public void testGetTableDescriptor() throws IOException { ADMIN.createTable(htd); Table table = TEST_UTIL.getConnection().getTable(htd.getTableName()); TableDescriptor confirmedHtd = table.getDescriptor(); + //HBASE-26246 introduced persist of store file tracker into table descriptor + htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACK_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()). + build(); assertEquals(0, TableDescriptor.COMPARATOR.compare(htd, confirmedHtd)); MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection()); table.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index 572a1d5a6226..a1832f49956a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -372,6 +374,10 @@ private void testCloneTableSchema(final TableName tableName, assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize()); assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled()); assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive()); + //HBASE-26246 introduced persist of store file tracker into table descriptor + tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACK_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()). + build(); TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc); if (preserveSplits) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java index 4a71baf44a13..cb89bf564720 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -147,6 +149,10 @@ public void testGetTableDescriptor() throws Exception { TableDescriptor desc = builder.build(); admin.createTable(desc).join(); TableDescriptor confirmedHtd = admin.getDescriptor(tableName).get(); + //HBASE-26246 introduced persist of store file tracker into table descriptor + desc = TableDescriptorBuilder.newBuilder(desc).setValue(TRACK_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()). + build(); assertEquals(0, TableDescriptor.COMPARATOR.compare(desc, confirmedHtd)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java index a3206432b810..2ceec5f241c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.procedure; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -60,6 +61,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -226,6 +228,11 @@ public static void validateTableCreation(final HMaster master, final TableName t assertTrue("family not found " + family[i], htd.getColumnFamily(Bytes.toBytes(family[i])) != null); } assertEquals(family.length, htd.getColumnFamilyCount()); + + // checks store file tracker impl has been properly set in htd + String storeFileTrackerImpl = + StoreFileTrackerFactory.getStoreFileTrackerImpl(master.getConfiguration()).getName(); + assertEquals(storeFileTrackerImpl, htd.getValue(TRACK_IMPL)); } public static void validateTableDeletion( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java index 3b57ed56744b..0bc77f0ef8d9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.master.procedure; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -38,6 +39,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; +import org.apache.hadoop.hbase.regionserver.storefiletracker.TestStoreFileTracker; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -88,6 +90,21 @@ private void testSimpleCreate(final TableName tableName, byte[][] splitKeys) thr MasterProcedureTestingUtility.validateTableCreation(getMaster(), tableName, regions, F1, F2); } + @Test + public void testCreateWithTrackImpl() throws Exception { + final TableName tableName = TableName.valueOf(name.getMethodName()); + ProcedureExecutor procExec = getMasterProcedureExecutor(); + TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1); + String trackerName = TestStoreFileTracker.class.getName(); + htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACK_IMPL, trackerName).build(); + RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null); + long procId = ProcedureTestingUtility.submitAndWait(procExec, + new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); + ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId)); + htd = getMaster().getTableDescriptors().get(tableName); + assertEquals(trackerName, htd.getValue(TRACK_IMPL)); + } + @Test public void testCreateWithoutColumnFamily() throws Exception { final ProcedureExecutor procExec = getMasterProcedureExecutor(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java index 05ca1fcb419b..1dc9c4e53fc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -27,6 +28,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,15 +40,21 @@ public class TestStoreFileTracker extends DefaultStoreFileTracker { public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { super(conf, isPrimaryReplica, ctx); - this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString(); - LOG.info("created storeId: {}", storeId); - trackedFiles.computeIfAbsent(storeId, v -> new ArrayList<>()); + if (ctx.getRegionFileSystem() != null) { + this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString(); + LOG.info("created storeId: {}", storeId); + trackedFiles.computeIfAbsent(storeId, v -> new ArrayList<>()); + } else { + LOG.info("ctx.getRegionFileSystem() returned null. Leaving storeId null."); + } + } @Override protected void doAddNewStoreFiles(Collection newFiles) throws IOException { LOG.info("adding to storeId: {}", storeId); trackedFiles.get(storeId).addAll(newFiles); + trackedFiles.putIfAbsent(storeId, (List)newFiles); } @Override From 434ec976ec8a7d7ac8670f04ee9e3fd8664f0cfb Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Tue, 14 Sep 2021 16:28:21 +0800 Subject: [PATCH 06/26] HBASE-26248 Should find a suitable way to let users specify the store file tracker implementation (#3665) Signed-off-by: Wellington Chevreuil --- .../MigrationStoreFileTracker.java | 13 ++- .../storefiletracker/StoreFileTracker.java | 7 +- .../StoreFileTrackerBase.java | 12 ++- .../StoreFileTrackerFactory.java | 97 +++++++++++++++++-- .../apache/hadoop/hbase/client/TestAdmin.java | 6 +- .../hadoop/hbase/client/TestAdmin3.java | 6 +- .../hbase/client/TestAsyncTableAdminApi.java | 6 +- .../hbase/client/TestAsyncTableAdminApi3.java | 6 +- .../MasterProcedureTestingUtility.java | 6 +- .../procedure/TestCreateTableProcedure.java | 6 +- .../TestMergesSplitsAddToTracker.java | 4 +- .../TestMigrationStoreFileTracker.java | 27 +++--- ...stRegionWithFileBasedStoreFileTracker.java | 3 +- .../TestStoreFileTrackerFactory.java | 58 +++++++++++ 14 files changed, 202 insertions(+), 55 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java index 483a240baded..3eeef9000576 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.yetus.audience.InterfaceAudience; @@ -44,8 +45,8 @@ class MigrationStoreFileTracker extends StoreFileTrackerBase { public MigrationStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { super(conf, isPrimaryReplica, ctx); - this.src = StoreFileTrackerFactory.create(conf, SRC_IMPL, isPrimaryReplica, ctx); - this.dst = StoreFileTrackerFactory.create(conf, DST_IMPL, isPrimaryReplica, ctx); + this.src = StoreFileTrackerFactory.createForMigration(conf, SRC_IMPL, isPrimaryReplica, ctx); + this.dst = StoreFileTrackerFactory.createForMigration(conf, DST_IMPL, isPrimaryReplica, ctx); Preconditions.checkArgument(!src.getClass().equals(dst.getClass()), "src and dst is the same: %s", src.getClass()); } @@ -90,7 +91,11 @@ void set(List files) { @Override public void persistConfiguration(TableDescriptorBuilder builder) { super.persistConfiguration(builder); - builder.setValue(SRC_IMPL, src.getClass().getName()); - builder.setValue(DST_IMPL, dst.getClass().getName()); + if (StringUtils.isEmpty(builder.getValue(SRC_IMPL))) { + builder.setValue(SRC_IMPL, src.getTrackerName()); + } + if (StringUtils.isEmpty(builder.getValue(DST_IMPL))) { + builder.setValue(DST_IMPL, dst.getTrackerName()); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java index 81fa1a9be5b2..59fe7ef52f96 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java @@ -75,7 +75,12 @@ void replace(Collection compactedFiles, Collection StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException; /** - * Saves StoreFileTracker implementations specific configs into the table descriptors. + * Saves StoreFileTracker implementations specific configurations into the table descriptors. + *

+ * This is used to avoid accidentally data loss when changing the cluster level store file tracker + * implementation, and also possible misconfiguration between master and region servers. + *

+ * See HBASE-26246 for more details. * @param builder The table descriptor builder for the given table. */ void persistConfiguration(TableDescriptorBuilder builder); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java index 83ebbc78ab35..a786add49b21 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver.storefiletracker; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import java.io.IOException; import java.util.Collection; @@ -84,13 +84,15 @@ public final void replace(Collection compactedFiles, @Override public void persistConfiguration(TableDescriptorBuilder builder) { - if (StringUtils.isEmpty(builder.getValue(TRACK_IMPL))) { - String trackerImpl = StoreFileTrackerFactory. - getStoreFileTrackerImpl(conf).getName(); - builder.setValue(TRACK_IMPL, trackerImpl).build(); + if (StringUtils.isEmpty(builder.getValue(TRACKER_IMPL))) { + builder.setValue(TRACKER_IMPL, getTrackerName()); } } + protected final String getTrackerName() { + return StoreFileTrackerFactory.getStoreFileTrackerName(getClass()); + } + private HFileContext createFileContext(Compression.Algorithm compression, boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context encryptionContext) { if (compression == null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index b9ec713cf235..9be19ec15ed8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -15,6 +15,9 @@ */ package org.apache.hadoop.hbase.regionserver.storefiletracker; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -33,22 +36,81 @@ /** * Factory method for creating store file tracker. + *

+ * The current implementations are: + *

    + *
  • default: DefaultStoreFileTracker, see {@link DefaultStoreFileTracker}.
  • + *
  • file:FileBasedStoreFileTracker, see {@link FileBasedStoreFileTracker}.
  • + *
  • migration:MigrationStoreFileTracker, see {@link MigrationStoreFileTracker}.
  • + *
+ * @see DefaultStoreFileTracker + * @see FileBasedStoreFileTracker + * @see MigrationStoreFileTracker */ -@InterfaceAudience.Private public final class StoreFileTrackerFactory { - public static final String TRACK_IMPL = "hbase.store.file-tracker.impl"; +@InterfaceAudience.Private +public final class StoreFileTrackerFactory { + private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerFactory.class); - public static Class getStoreFileTrackerImpl(Configuration conf) { - return conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class); + public static final String TRACKER_IMPL = "hbase.store.file-tracker.impl"; + + /** + * Maps between configuration names for trackers and implementation classes. + */ + public enum Trackers { + DEFAULT(DefaultStoreFileTracker.class), FILE(FileBasedStoreFileTracker.class), + MIGRATION(MigrationStoreFileTracker.class); + + final Class clazz; + + Trackers(Class clazz) { + this.clazz = clazz; + } + } + + private static final Map, Trackers> CLASS_TO_ENUM = reverse(); + + private static Map, Trackers> reverse() { + Map, Trackers> map = new HashMap<>(); + for (Trackers tracker : Trackers.values()) { + map.put(tracker.clazz, tracker); + } + return Collections.unmodifiableMap(map); + } + + private StoreFileTrackerFactory() { + } + + public static String getStoreFileTrackerName(Configuration conf) { + return conf.get(TRACKER_IMPL, Trackers.DEFAULT.name()); + } + + static String getStoreFileTrackerName(Class clazz) { + Trackers name = CLASS_TO_ENUM.get(clazz); + return name != null ? name.name() : clazz.getName(); + } + + private static Class getTrackerClass(Configuration conf) { + try { + Trackers tracker = Trackers.valueOf(getStoreFileTrackerName(conf).toUpperCase()); + return tracker.clazz; + } catch (IllegalArgumentException e) { + // Fall back to them specifying a class name + return conf.getClass(TRACKER_IMPL, Trackers.DEFAULT.clazz, StoreFileTracker.class); + } } public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { - Class tracker = getStoreFileTrackerImpl(conf); + Class tracker = getTrackerClass(conf); LOG.info("instantiating StoreFileTracker impl {}", tracker.getName()); return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); } + /** + * Used at master side when splitting/merging regions, as we do not have a Store, thus no + * StoreContext at master side. + */ public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, String family, HRegionFileSystem regionFs) { ColumnFamilyDescriptorBuilder fDescBuilder = @@ -63,15 +125,30 @@ public static Configuration mergeConfigurations(Configuration global, TableDescr return StoreUtils.createStoreConfiguration(global, table, family); } - static StoreFileTrackerBase create(Configuration conf, String configName, + /** + * Create store file tracker to be used as source or destination for + * {@link MigrationStoreFileTracker}. + */ + static StoreFileTrackerBase createForMigration(Configuration conf, String configName, boolean isPrimaryReplica, StoreContext ctx) { - String className = + String trackerName = Preconditions.checkNotNull(conf.get(configName), "config %s is not set", configName); Class tracker; try { - tracker = Class.forName(className).asSubclass(StoreFileTrackerBase.class); - } catch (ClassNotFoundException e) { - throw new RuntimeException(e); + tracker = + Trackers.valueOf(trackerName.toUpperCase()).clazz.asSubclass(StoreFileTrackerBase.class); + } catch (IllegalArgumentException e) { + // Fall back to them specifying a class name + try { + tracker = Class.forName(trackerName).asSubclass(StoreFileTrackerBase.class); + } catch (ClassNotFoundException cnfe) { + throw new RuntimeException(cnfe); + } + } + // prevent nest of MigrationStoreFileTracker, it will cause infinite recursion. + if (MigrationStoreFileTracker.class.isAssignableFrom(tracker)) { + throw new IllegalArgumentException("Should not specify " + configName + " as " + + Trackers.MIGRATION + " because it can not be nested"); } LOG.info("instantiating StoreFileTracker impl {} as {}", tracker.getName(), configName); return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 10ab4961a5b4..ba843fe43460 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.HBaseTestingUtil.countRows; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -426,8 +426,8 @@ private void testCloneTableSchema(final TableName tableName, final TableName new assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled()); assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive()); // HBASE-26246 introduced persist of store file tracker into table descriptor - tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACK_IMPL, - StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()). + tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACKER_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())). build(); TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java index 7d40fd12bda5..c2de0fbd3555 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -239,8 +239,8 @@ public void testGetTableDescriptor() throws IOException { Table table = TEST_UTIL.getConnection().getTable(htd.getTableName()); TableDescriptor confirmedHtd = table.getDescriptor(); //HBASE-26246 introduced persist of store file tracker into table descriptor - htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACK_IMPL, - StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()). + htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACKER_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())). build(); assertEquals(0, TableDescriptor.COMPARATOR.compare(htd, confirmedHtd)); MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index a1832f49956a..147630b53a38 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -375,8 +375,8 @@ private void testCloneTableSchema(final TableName tableName, assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled()); assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive()); //HBASE-26246 introduced persist of store file tracker into table descriptor - tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACK_IMPL, - StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()). + tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACKER_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())). build(); TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java index cb89bf564720..67377b07c180 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -150,8 +150,8 @@ public void testGetTableDescriptor() throws Exception { admin.createTable(desc).join(); TableDescriptor confirmedHtd = admin.getDescriptor(tableName).get(); //HBASE-26246 introduced persist of store file tracker into table descriptor - desc = TableDescriptorBuilder.newBuilder(desc).setValue(TRACK_IMPL, - StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()). + desc = TableDescriptorBuilder.newBuilder(desc).setValue(TRACKER_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())). build(); assertEquals(0, TableDescriptor.COMPARATOR.compare(desc, confirmedHtd)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java index 2ceec5f241c6..726e117ef1ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.master.procedure; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -231,8 +231,8 @@ public static void validateTableCreation(final HMaster master, final TableName t // checks store file tracker impl has been properly set in htd String storeFileTrackerImpl = - StoreFileTrackerFactory.getStoreFileTrackerImpl(master.getConfiguration()).getName(); - assertEquals(storeFileTrackerImpl, htd.getValue(TRACK_IMPL)); + StoreFileTrackerFactory.getStoreFileTrackerName(master.getConfiguration()); + assertEquals(storeFileTrackerImpl, htd.getValue(TRACKER_IMPL)); } public static void validateTableDeletion( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java index 0bc77f0ef8d9..f432c8060d3d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.master.procedure; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -96,13 +96,13 @@ public void testCreateWithTrackImpl() throws Exception { ProcedureExecutor procExec = getMasterProcedureExecutor(); TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1); String trackerName = TestStoreFileTracker.class.getName(); - htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACK_IMPL, trackerName).build(); + htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACKER_IMPL, trackerName).build(); RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null); long procId = ProcedureTestingUtility.submitAndWait(procExec, new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId)); htd = getMaster().getTableDescriptors().get(tableName); - assertEquals(trackerName, htd.getValue(TRACK_IMPL)); + assertEquals(trackerName, htd.getValue(TRACKER_IMPL)); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java index c6205cb18493..68fc444493c4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory. - TRACK_IMPL; + TRACKER_IMPL; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -74,7 +74,7 @@ public class TestMergesSplitsAddToTracker { @BeforeClass public static void setupClass() throws Exception { - TEST_UTIL.getConfiguration().set(TRACK_IMPL, TestStoreFileTracker.class.getName()); + TEST_UTIL.getConfiguration().set(TRACKER_IMPL, TestStoreFileTracker.class.getName()); TEST_UTIL.startMiniCluster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java index e9ebb4c792e4..119f8fb553f8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; @@ -86,10 +85,10 @@ public class TestMigrationStoreFileTracker { public TestName name = new TestName(); @Parameter(0) - public Class srcImplClass; + public StoreFileTrackerFactory.Trackers srcImpl; @Parameter(1) - public Class dstImplClass; + public StoreFileTrackerFactory.Trackers dstImpl; private HRegion region; @@ -99,11 +98,13 @@ public class TestMigrationStoreFileTracker { @Parameters(name = "{index}: src={0}, dst={1}") public static List params() { - List> impls = - Arrays.asList(DefaultStoreFileTracker.class, FileBasedStoreFileTracker.class); List params = new ArrayList<>(); - for (Class src : impls) { - for (Class dst : impls) { + for (StoreFileTrackerFactory.Trackers src : StoreFileTrackerFactory.Trackers.values()) { + for (StoreFileTrackerFactory.Trackers dst : StoreFileTrackerFactory.Trackers.values()) { + if (src == StoreFileTrackerFactory.Trackers.MIGRATION + || dst == StoreFileTrackerFactory.Trackers.MIGRATION) { + continue; + } if (src.equals(dst)) { continue; } @@ -122,8 +123,8 @@ public static void setUpBeforeClass() { @Before public void setUp() throws IOException { Configuration conf = UTIL.getConfiguration(); - conf.setClass(MigrationStoreFileTracker.SRC_IMPL, srcImplClass, StoreFileTrackerBase.class); - conf.setClass(MigrationStoreFileTracker.DST_IMPL, dstImplClass, StoreFileTrackerBase.class); + conf.set(MigrationStoreFileTracker.SRC_IMPL, srcImpl.name().toLowerCase()); + conf.set(MigrationStoreFileTracker.DST_IMPL, dstImpl.name().toLowerCase()); rootDir = UTIL.getDataTestDir(name.getMethodName().replaceAll("[=:\\[ ]", "_")); wal = HBaseTestingUtil.createWal(conf, rootDir, RI); } @@ -145,7 +146,7 @@ private List getStoreFiles() { private HRegion createRegion(Class trackerImplClass) throws IOException { Configuration conf = new Configuration(UTIL.getConfiguration()); - conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, trackerImplClass, StoreFileTracker.class); + conf.setClass(StoreFileTrackerFactory.TRACKER_IMPL, trackerImplClass, StoreFileTracker.class); return HRegion.createHRegion(RI, rootDir, conf, TD, wal, true); } @@ -155,7 +156,7 @@ private void reopenRegion(Class trackerImplClass List before = getStoreFiles(); region.close(); Configuration conf = new Configuration(UTIL.getConfiguration()); - conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, trackerImplClass, StoreFileTracker.class); + conf.setClass(StoreFileTrackerFactory.TRACKER_IMPL, trackerImplClass, StoreFileTracker.class); region = HRegion.openHRegion(rootDir, RI, TD, wal, conf); List after = getStoreFiles(); assertEquals(before.size(), after.size()); @@ -180,14 +181,14 @@ private void verifyData(int start, int end) throws IOException { @Test public void testMigration() throws IOException { - region = createRegion(srcImplClass); + region = createRegion(srcImpl.clazz.asSubclass(StoreFileTrackerBase.class)); putData(0, 100); verifyData(0, 100); reopenRegion(MigrationStoreFileTracker.class); verifyData(0, 100); region.compact(true); putData(100, 200); - reopenRegion(dstImplClass); + reopenRegion(dstImpl.clazz.asSubclass(StoreFileTrackerBase.class)); verifyData(0, 200); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java index 3bc60d13fd94..77cd3c448fa0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java @@ -71,8 +71,7 @@ public class TestRegionWithFileBasedStoreFileTracker { @Before public void setUp() throws IOException { Configuration conf = new Configuration(UTIL.getConfiguration()); - conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, FileBasedStoreFileTracker.class, - StoreFileTracker.class); + conf.set(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()); region = HBaseTestingUtil.createRegionAndWAL(RI, UTIL.getDataTestDir(name.getMethodName()), conf, TD); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java new file mode 100644 index 000000000000..41f2afdfa421 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java @@ -0,0 +1,58 @@ +/** + * 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.storefiletracker; + +import static org.junit.Assert.assertThrows; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestStoreFileTrackerFactory { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStoreFileTrackerFactory.class); + + @Test + public void testCreateForMigration() { + Configuration conf = HBaseConfiguration.create(); + String configName = "config"; + + // no config + assertThrows(NullPointerException.class, () -> StoreFileTrackerFactory.createForMigration(conf, + configName, false, StoreContext.getBuilder().build())); + + // class not found + conf.set(configName, "config"); + assertThrows(RuntimeException.class, () -> StoreFileTrackerFactory.createForMigration(conf, + configName, false, StoreContext.getBuilder().build())); + + // nested MigrationStoreFileTracker + conf.setClass(configName, MigrationStoreFileTracker.class, StoreFileTrackerBase.class); + assertThrows(IllegalArgumentException.class, () -> StoreFileTrackerFactory + .createForMigration(conf, configName, false, StoreContext.getBuilder().build())); + } +} From 70906cff2f33ffce91a3e0128e057be0792f66e8 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 15 Sep 2021 23:00:03 +0800 Subject: [PATCH 07/26] HBASE-26264 Add more checks to prevent misconfiguration on store file tracker (#3681) Signed-off-by: Josh Elser --- .../MergeTableRegionsProcedure.java | 3 +- .../assignment/SplitTableRegionProcedure.java | 3 +- .../procedure/CreateTableProcedure.java | 8 +- .../procedure/ModifyTableProcedure.java | 5 + .../hbase/regionserver/HRegionFileSystem.java | 2 +- .../MigrationStoreFileTracker.java | 8 + .../StoreFileTrackerFactory.java | 173 ++++++++++++- .../TestChangeStoreFileTracker.java | 242 ++++++++++++++++++ 8 files changed, 422 insertions(+), 22 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestChangeStoreFileTracker.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index e6bbe445d077..e9051dab0913 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -614,8 +614,7 @@ private List mergeStoreFiles(MasterProcedureEnv env, HRegionFileSystem reg String family = hcd.getNameAsString(); Configuration trackerConfig = StoreFileTrackerFactory.mergeConfigurations(env.getMasterConfiguration(), htd, hcd); - StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, true, - family, regionFs); + StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, family, regionFs); final Collection storeFiles = tracker.load(); if (storeFiles != null && storeFiles.size() > 0) { final Configuration storeConfiguration = diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index ff16dc5514b8..aa0c938fa21f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -670,8 +670,7 @@ private Pair, List> splitStoreFiles(final MasterProcedureEnv en String family = cfd.getNameAsString(); Configuration trackerConfig = StoreFileTrackerFactory. mergeConfigurations(env.getMasterConfiguration(), htd, htd.getColumnFamily(cfd.getName())); - StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, true, - family, regionFs); + StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, family, regionFs); Collection sfis = tracker.load(); if (sfis == null) { continue; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java index dccea5512c1f..ee8e51ffe86b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java @@ -277,15 +277,17 @@ private boolean prepareCreate(final MasterProcedureEnv env) throws IOException { MasterProcedureUtil.checkGroupNotEmpty(rsGroupInfo, forWhom); } + // check for store file tracker configurations + StoreFileTrackerFactory.checkForCreateTable(env.getMasterConfiguration(), tableDescriptor); + return true; } private void preCreate(final MasterProcedureEnv env) throws IOException, InterruptedException { if (!getTableName().isSystemTable()) { - ProcedureSyncWait.getMasterQuotaManager(env) - .checkNamespaceTableAndRegionQuota( - getTableName(), (newRegions != null ? newRegions.size() : 0)); + ProcedureSyncWait.getMasterQuotaManager(env).checkNamespaceTableAndRegionQuota(getTableName(), + (newRegions != null ? newRegions.size() : 0)); } TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index 247dd9c202f4..1640644328f0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.rsgroup.RSGroupInfo; import org.apache.hadoop.hbase.util.Bytes; @@ -325,6 +326,10 @@ private void prepareModify(final MasterProcedureEnv env) throws IOException { modifiedTableDescriptor.getRegionServerGroup(), forWhom); MasterProcedureUtil.checkGroupNotEmpty(rsGroupInfo, forWhom); } + + // check for store file tracker configurations + StoreFileTrackerFactory.checkForModifyTable(env.getMasterConfiguration(), + unmodifiedTableDescriptor, modifiedTableDescriptor); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index aa0ee273ea9e..e78d8ad8aa1e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -629,7 +629,7 @@ private void insertRegionFilesIntoStoreTracker(List allFiles, MasterProced Configuration config = StoreFileTrackerFactory.mergeConfigurations(conf, tblDesc, tblDesc.getColumnFamily(Bytes.toBytes(familyName))); return StoreFileTrackerFactory. - create(config, true, familyName, regionFs); + create(config, familyName, regionFs); }); fileInfoMap.computeIfAbsent(familyName, l -> new ArrayList<>()); List infos = fileInfoMap.get(familyName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java index 3eeef9000576..1946d4bf7eef 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java @@ -98,4 +98,12 @@ public void persistConfiguration(TableDescriptorBuilder builder) { builder.setValue(DST_IMPL, dst.getTrackerName()); } } + + static Class getSrcTrackerClass(Configuration conf) { + return StoreFileTrackerFactory.getStoreFileTrackerClassForMigration(conf, SRC_IMPL); + } + + static Class getDstTrackerClass(Configuration conf) { + return StoreFileTrackerFactory.getStoreFileTrackerClassForMigration(conf, DST_IMPL); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index 9be19ec15ed8..90704fe528a0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -15,10 +15,12 @@ */ package org.apache.hadoop.hbase.regionserver.storefiletracker; +import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -111,13 +113,13 @@ public static StoreFileTracker create(Configuration conf, boolean isPrimaryRepli * Used at master side when splitting/merging regions, as we do not have a Store, thus no * StoreContext at master side. */ - public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, String family, + public static StoreFileTracker create(Configuration conf, String family, HRegionFileSystem regionFs) { ColumnFamilyDescriptorBuilder fDescBuilder = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family)); StoreContext ctx = StoreContext.getBuilder().withColumnFamilyDescriptor(fDescBuilder.build()) .withRegionFileSystem(regionFs).build(); - return StoreFileTrackerFactory.create(conf, isPrimaryReplica, ctx); + return StoreFileTrackerFactory.create(conf, true, ctx); } public static Configuration mergeConfigurations(Configuration global, TableDescriptor table, @@ -125,30 +127,35 @@ public static Configuration mergeConfigurations(Configuration global, TableDescr return StoreUtils.createStoreConfiguration(global, table, family); } - /** - * Create store file tracker to be used as source or destination for - * {@link MigrationStoreFileTracker}. - */ - static StoreFileTrackerBase createForMigration(Configuration conf, String configName, - boolean isPrimaryReplica, StoreContext ctx) { + static Class + getStoreFileTrackerClassForMigration(Configuration conf, String configName) { String trackerName = Preconditions.checkNotNull(conf.get(configName), "config %s is not set", configName); - Class tracker; try { - tracker = - Trackers.valueOf(trackerName.toUpperCase()).clazz.asSubclass(StoreFileTrackerBase.class); + return Trackers.valueOf(trackerName.toUpperCase()).clazz + .asSubclass(StoreFileTrackerBase.class); } catch (IllegalArgumentException e) { // Fall back to them specifying a class name try { - tracker = Class.forName(trackerName).asSubclass(StoreFileTrackerBase.class); + return Class.forName(trackerName).asSubclass(StoreFileTrackerBase.class); } catch (ClassNotFoundException cnfe) { throw new RuntimeException(cnfe); } } + } + + /** + * Create store file tracker to be used as source or destination for + * {@link MigrationStoreFileTracker}. + */ + static StoreFileTrackerBase createForMigration(Configuration conf, String configName, + boolean isPrimaryReplica, StoreContext ctx) { + Class tracker = + getStoreFileTrackerClassForMigration(conf, configName); // prevent nest of MigrationStoreFileTracker, it will cause infinite recursion. if (MigrationStoreFileTracker.class.isAssignableFrom(tracker)) { - throw new IllegalArgumentException("Should not specify " + configName + " as " + - Trackers.MIGRATION + " because it can not be nested"); + throw new IllegalArgumentException("Should not specify " + configName + " as " + + Trackers.MIGRATION + " because it can not be nested"); } LOG.info("instantiating StoreFileTracker impl {} as {}", tracker.getName(), configName); return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); @@ -161,4 +168,142 @@ public static void persistTrackerConfig(Configuration conf, TableDescriptorBuild StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, context); tracker.persistConfiguration(builder); } + + // should not use MigrationStoreFileTracker for new family + private static void checkForNewFamily(Configuration conf, TableDescriptor table, + ColumnFamilyDescriptor family) throws IOException { + Configuration mergedConf = mergeConfigurations(conf, table, family); + Class tracker = getTrackerClass(mergedConf); + if (MigrationStoreFileTracker.class.isAssignableFrom(tracker)) { + throw new DoNotRetryIOException( + "Should not use " + Trackers.MIGRATION + " as store file tracker for new family " + + family.getNameAsString() + " of table " + table.getTableName()); + } + } + + /** + * Pre check when creating a new table. + *

+ * For now, only make sure that we do not use {@link Trackers#MIGRATION} for newly created tables. + * @throws IOException when there are check errors, the upper layer should fail the + * {@code CreateTableProcedure}. + */ + public static void checkForCreateTable(Configuration conf, TableDescriptor table) + throws IOException { + for (ColumnFamilyDescriptor family : table.getColumnFamilies()) { + checkForNewFamily(conf, table, family); + } + } + + + /** + * Pre check when modifying a table. + *

+ * The basic idea is when you want to change the store file tracker implementation, you should use + * {@link Trackers#MIGRATION} first and then change to the destination store file tracker + * implementation. + *

+ * There are several rules: + *

    + *
  • For newly added family, you should not use {@link Trackers#MIGRATION}.
  • + *
  • For modifying a family: + *
      + *
    • If old tracker is {@link Trackers#MIGRATION}, then: + *
        + *
      • The new tracker is also {@link Trackers#MIGRATION}, then they must have the same src and + * dst tracker.
      • + *
      • The new tracker is not {@link Trackers#MIGRATION}, then the new tracker must be the dst + * tracker of the old tracker.
      • + *
      + *
    • + *
    • If the old tracker is not {@link Trackers#MIGRATION}, then: + *
        + *
      • If the new tracker is {@link Trackers#MIGRATION}, then the old tracker must be the src + * tracker of the new tracker.
      • + *
      • If the new tracker is not {@link Trackers#MIGRATION}, then the new tracker must be the same + * with old tracker.
      • + *
      + *
    • + *
    + *
  • + *
+ * @throws IOException when there are check errors, the upper layer should fail the + * {@code ModifyTableProcedure}. + */ + public static void checkForModifyTable(Configuration conf, TableDescriptor oldTable, + TableDescriptor newTable) throws IOException { + for (ColumnFamilyDescriptor newFamily : newTable.getColumnFamilies()) { + ColumnFamilyDescriptor oldFamily = oldTable.getColumnFamily(newFamily.getName()); + if (oldFamily == null) { + checkForNewFamily(conf, newTable, newFamily); + continue; + } + Configuration oldConf = mergeConfigurations(conf, oldTable, oldFamily); + Configuration newConf = mergeConfigurations(conf, newTable, newFamily); + + Class oldTracker = getTrackerClass(oldConf); + Class newTracker = getTrackerClass(newConf); + + if (MigrationStoreFileTracker.class.isAssignableFrom(oldTracker)) { + Class oldSrcTracker = + MigrationStoreFileTracker.getSrcTrackerClass(oldConf); + Class oldDstTracker = + MigrationStoreFileTracker.getDstTrackerClass(oldConf); + if (oldTracker.equals(newTracker)) { + // confirm that we have the same src tracker and dst tracker + Class newSrcTracker = + MigrationStoreFileTracker.getSrcTrackerClass(newConf); + if (!oldSrcTracker.equals(newSrcTracker)) { + throw new DoNotRetryIOException( + "The src tracker has been changed from " + getStoreFileTrackerName(oldSrcTracker) + + " to " + getStoreFileTrackerName(newSrcTracker) + " for family " + + newFamily.getNameAsString() + " of table " + newTable.getTableName()); + } + Class newDstTracker = + MigrationStoreFileTracker.getDstTrackerClass(newConf); + if (!oldDstTracker.equals(newDstTracker)) { + throw new DoNotRetryIOException( + "The dst tracker has been changed from " + getStoreFileTrackerName(oldDstTracker) + + " to " + getStoreFileTrackerName(newDstTracker) + " for family " + + newFamily.getNameAsString() + " of table " + newTable.getTableName()); + } + } else { + // we can only change to the dst tracker + if (!newTracker.equals(oldDstTracker)) { + throw new DoNotRetryIOException( + "Should migrate tracker to " + getStoreFileTrackerName(oldDstTracker) + " but got " + + getStoreFileTrackerName(newTracker) + " for family " + newFamily.getNameAsString() + + " of table " + newTable.getTableName()); + } + } + } else { + if (!oldTracker.equals(newTracker)) { + // can only change to MigrationStoreFileTracker and the src tracker should be the old + // tracker + if (!MigrationStoreFileTracker.class.isAssignableFrom(newTracker)) { + throw new DoNotRetryIOException("Should change to " + Trackers.MIGRATION + + " first when migrating from " + getStoreFileTrackerName(oldTracker) + " for family " + + newFamily.getNameAsString() + " of table " + newTable.getTableName()); + } + Class newSrcTracker = + MigrationStoreFileTracker.getSrcTrackerClass(newConf); + if (!oldTracker.equals(newSrcTracker)) { + throw new DoNotRetryIOException( + "Should use src tracker " + getStoreFileTrackerName(oldTracker) + " first but got " + + getStoreFileTrackerName(newSrcTracker) + " when migrating from " + + getStoreFileTrackerName(oldTracker) + " for family " + newFamily.getNameAsString() + + " of table " + newTable.getTableName()); + } + Class newDstTracker = + MigrationStoreFileTracker.getDstTrackerClass(newConf); + // the src and dst tracker should not be the same + if (newSrcTracker.equals(newDstTracker)) { + throw new DoNotRetryIOException("The src tracker and dst tracker are both " + + getStoreFileTrackerName(newSrcTracker) + " for family " + + newFamily.getNameAsString() + " of table " + newTable.getTableName()); + } + } + } + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestChangeStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestChangeStoreFileTracker.java new file mode 100644 index 000000000000..70f62c02ed28 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestChangeStoreFileTracker.java @@ -0,0 +1,242 @@ +/** + * 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.storefiletracker; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; + +/** + * Test changing store file tracker implementation by altering table. + */ +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestChangeStoreFileTracker { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChangeStoreFileTracker.class); + + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + + @Rule + public final TableNameTestRule tableName = new TableNameTestRule(); + + @BeforeClass + public static void setUp() throws Exception { + UTIL.startMiniCluster(1); + } + + @AfterClass + public static void tearDown() throws IOException { + UTIL.shutdownMiniCluster(); + } + + @Test(expected = DoNotRetryIOException.class) + public void testCreateError() throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .build(); + UTIL.getAdmin().createTable(td); + } + + @Test(expected = DoNotRetryIOException.class) + public void testModifyError1() throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build(); + UTIL.getAdmin().createTable(td); + TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .build(); + UTIL.getAdmin().modifyTable(newTd); + } + + @Test(expected = DoNotRetryIOException.class) + public void testModifyError2() throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build(); + UTIL.getAdmin().createTable(td); + TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .build(); + UTIL.getAdmin().modifyTable(newTd); + } + + @Test(expected = DoNotRetryIOException.class) + public void testModifyError3() throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build(); + UTIL.getAdmin().createTable(td); + TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .build(); + UTIL.getAdmin().modifyTable(newTd); + } + + // return the TableDescriptor for creating table + private TableDescriptor createTableAndChangeToMigrationTracker() throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build(); + UTIL.getAdmin().createTable(td); + TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .build(); + UTIL.getAdmin().modifyTable(newTd); + return td; + } + + @Test(expected = DoNotRetryIOException.class) + public void testModifyError4() throws IOException { + TableDescriptor td = createTableAndChangeToMigrationTracker(); + TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .build(); + UTIL.getAdmin().modifyTable(newTd); + } + + @Test(expected = DoNotRetryIOException.class) + public void testModifyError5() throws IOException { + TableDescriptor td = createTableAndChangeToMigrationTracker(); + TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .build(); + UTIL.getAdmin().modifyTable(newTd); + } + + @Test(expected = DoNotRetryIOException.class) + public void testModifyError6() throws IOException { + TableDescriptor td = createTableAndChangeToMigrationTracker(); + TableDescriptor newTd = + TableDescriptorBuilder.newBuilder(td).setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.DEFAULT.name()).build(); + UTIL.getAdmin().modifyTable(newTd); + } + + @Test(expected = DoNotRetryIOException.class) + public void testModifyError7() throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build(); + UTIL.getAdmin().createTable(td); + TableDescriptor newTd = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("family1")) + .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .build()) + .build(); + UTIL.getAdmin().modifyTable(newTd); + } + + // actually a NPE as we do not specify the src and dst impl for migration store file tracker + @Test(expected = IOException.class) + public void testModifyError8() throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build(); + UTIL.getAdmin().createTable(td); + TableDescriptor newTd = + TableDescriptorBuilder.newBuilder(td).setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()).build(); + UTIL.getAdmin().modifyTable(newTd); + } + + private String getStoreFileName(TableName table, byte[] family) { + return Iterables + .getOnlyElement(Iterables.getOnlyElement(UTIL.getMiniHBaseCluster().getRegions(table)) + .getStore(family).getStorefiles()) + .getPath().getName(); + } + + @Test + public void testModify() throws IOException { + TableName tn = tableName.getTableName(); + byte[] row = Bytes.toBytes("row"); + byte[] family = Bytes.toBytes("family"); + byte[] qualifier = Bytes.toBytes("qualifier"); + byte[] value = Bytes.toBytes("value"); + TableDescriptor td = TableDescriptorBuilder.newBuilder(tn) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build(); + UTIL.getAdmin().createTable(td); + try (Table table = UTIL.getConnection().getTable(tn)) { + table.put(new Put(row).addColumn(family, qualifier, value)); + } + UTIL.flush(tn); + String fileName = getStoreFileName(tn, family); + + TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .build(); + UTIL.getAdmin().modifyTable(newTd); + assertEquals(fileName, getStoreFileName(tn, family)); + try (Table table = UTIL.getConnection().getTable(tn)) { + assertArrayEquals(value, table.get(new Get(row)).getValue(family, qualifier)); + } + + TableDescriptor newTd2 = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .build(); + UTIL.getAdmin().modifyTable(newTd2); + assertEquals(fileName, getStoreFileName(tn, family)); + try (Table table = UTIL.getConnection().getTable(tn)) { + assertArrayEquals(value, table.get(new Get(row)).getValue(family, qualifier)); + } + } +} From c705d80b54c354c3967e823cc32ceca09a3a57d2 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 17 Sep 2021 09:40:44 +0800 Subject: [PATCH 08/26] HBASE-26280 Use store file tracker when snapshoting (#3685) Signed-off-by: Wellington Chevreuil Reviewed-by: Josh Elser --- .../MergeTableRegionsProcedure.java | 5 +- .../assignment/SplitTableRegionProcedure.java | 5 +- .../hbase/regionserver/HRegionFileSystem.java | 10 +-- .../StoreFileTrackerFactory.java | 17 ++--- .../hbase/snapshot/SnapshotManifest.java | 42 +++++------ ...apshotFromClientCloneLinksAfterDelete.java | 4 +- .../client/TestMobSnapshotFromClient.java | 7 +- .../hbase/client/TestSnapshotFromClient.java | 30 ++++++-- .../hbase/regionserver/TestHStoreFile.java | 6 +- .../snapshot/MobSnapshotTestingUtils.java | 74 ++++++++++--------- .../hbase/snapshot/SnapshotTestingUtils.java | 16 ++-- 11 files changed, 107 insertions(+), 109 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index e9051dab0913..0f41db5cad88 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -612,9 +612,8 @@ private List mergeStoreFiles(MasterProcedureEnv env, HRegionFileSystem reg List mergedFiles = new ArrayList<>(); for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { String family = hcd.getNameAsString(); - Configuration trackerConfig = - StoreFileTrackerFactory.mergeConfigurations(env.getMasterConfiguration(), htd, hcd); - StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, family, regionFs); + StoreFileTracker tracker = + StoreFileTrackerFactory.create(env.getMasterConfiguration(), htd, hcd, regionFs); final Collection storeFiles = tracker.load(); if (storeFiles != null && storeFiles.size() > 0) { final Configuration storeConfiguration = diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index aa0c938fa21f..effdba4f2012 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -668,9 +668,8 @@ private Pair, List> splitStoreFiles(final MasterProcedureEnv en new HashMap>(htd.getColumnFamilyCount()); for (ColumnFamilyDescriptor cfd : htd.getColumnFamilies()) { String family = cfd.getNameAsString(); - Configuration trackerConfig = StoreFileTrackerFactory. - mergeConfigurations(env.getMasterConfiguration(), htd, htd.getColumnFamily(cfd.getName())); - StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, family, regionFs); + StoreFileTracker tracker = + StoreFileTrackerFactory.create(env.getMasterConfiguration(), htd, cfd, regionFs); Collection sfis = tracker.load(); if (sfis == null) { continue; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index e78d8ad8aa1e..811002561f0a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -598,7 +598,6 @@ void cleanupDaughterRegion(final RegionInfo regionInfo) throws IOException { * to the proper location in the filesystem. * * @param regionInfo daughter {@link org.apache.hadoop.hbase.client.RegionInfo} - * @throws IOException */ public Path commitDaughterRegion(final RegionInfo regionInfo, List allRegionFiles, MasterProcedureEnv env) throws IOException { @@ -625,12 +624,8 @@ private void insertRegionFilesIntoStoreTracker(List allFiles, MasterProced Map> fileInfoMap = new HashMap<>(); for(Path file : allFiles) { String familyName = file.getParent().getName(); - trackerMap.computeIfAbsent(familyName, t -> { - Configuration config = StoreFileTrackerFactory.mergeConfigurations(conf, tblDesc, - tblDesc.getColumnFamily(Bytes.toBytes(familyName))); - return StoreFileTrackerFactory. - create(config, familyName, regionFs); - }); + trackerMap.computeIfAbsent(familyName, t -> StoreFileTrackerFactory.create(conf, tblDesc, + tblDesc.getColumnFamily(Bytes.toBytes(familyName)), regionFs)); fileInfoMap.computeIfAbsent(familyName, l -> new ArrayList<>()); List infos = fileInfoMap.get(familyName); infos.add(new StoreFileInfo(conf, fs, file, true)); @@ -676,7 +671,6 @@ public void createSplitsDir(RegionInfo daughterA, RegionInfo daughterB) throws I * this method is invoked on the Master side, then the RegionSplitPolicy will * NOT have a reference to a Region. * @return Path to created reference. - * @throws IOException */ public Path splitStoreFile(RegionInfo hri, String familyName, HStoreFile f, byte[] splitRow, boolean top, RegionSplitPolicy splitPolicy) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index 90704fe528a0..b586027f8333 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -22,13 +22,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreUtils; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -113,16 +111,15 @@ public static StoreFileTracker create(Configuration conf, boolean isPrimaryRepli * Used at master side when splitting/merging regions, as we do not have a Store, thus no * StoreContext at master side. */ - public static StoreFileTracker create(Configuration conf, String family, - HRegionFileSystem regionFs) { - ColumnFamilyDescriptorBuilder fDescBuilder = - ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family)); - StoreContext ctx = StoreContext.getBuilder().withColumnFamilyDescriptor(fDescBuilder.build()) - .withRegionFileSystem(regionFs).build(); - return StoreFileTrackerFactory.create(conf, true, ctx); + public static StoreFileTracker create(Configuration conf, TableDescriptor td, + ColumnFamilyDescriptor cfd, HRegionFileSystem regionFs) { + StoreContext ctx = + StoreContext.getBuilder().withColumnFamilyDescriptor(cfd).withRegionFileSystem(regionFs) + .withFamilyStoreDirectoryPath(regionFs.getStoreDir(cfd.getNameAsString())).build(); + return StoreFileTrackerFactory.create(mergeConfigurations(conf, td, cfd), true, ctx); } - public static Configuration mergeConfigurations(Configuration global, TableDescriptor table, + private static Configuration mergeConfigurations(Configuration global, TableDescriptor table, ColumnFamilyDescriptor family) { return StoreUtils.createStoreConfiguration(global, table, family); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java index 5e82cad6b494..f154aa92cd6e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java @@ -47,7 +47,8 @@ import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.Threads; @@ -291,8 +292,8 @@ public void addRegion(final Path tableDir, final RegionInfo regionInfo) throws I addRegion(tableDir, regionInfo, visitor); } - protected void addRegion(final Path tableDir, final RegionInfo regionInfo, RegionVisitor visitor) - throws IOException { + protected void addRegion(Path tableDir, RegionInfo regionInfo, RegionVisitor visitor) + throws IOException { boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo); try { Path baseDir = tableDir; @@ -300,8 +301,8 @@ protected void addRegion(final Path tableDir, final RegionInfo regionInfo, Regio if (isMobRegion) { baseDir = CommonFSUtils.getTableDir(MobUtils.getMobHome(conf), regionInfo.getTable()); } - HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, rootFs, - baseDir, regionInfo, true); + HRegionFileSystem regionFs = + HRegionFileSystem.openRegionFromFileSystem(conf, rootFs, baseDir, regionInfo, true); monitor.rethrowException(); // 1. dump region meta info into the snapshot directory @@ -317,26 +318,19 @@ protected void addRegion(final Path tableDir, final RegionInfo regionInfo, Regio // in batches and may miss files being added/deleted. This could be more robust (iteratively // checking to see if we have all the files until we are sure), but the limit is currently // 1000 files/batch, far more than the number of store files under a single column family. - Collection familyNames = regionFs.getFamilies(); - if (familyNames != null) { - for (String familyName: familyNames) { - Object familyData = visitor.familyOpen(regionData, Bytes.toBytes(familyName)); - monitor.rethrowException(); - - Collection storeFiles = regionFs.getStoreFiles(familyName); - if (storeFiles == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("No files under family: " + familyName); - } - continue; - } - - // 2.1. build the snapshot reference for the store - // iterate through all the store's files and create "references". - addReferenceFiles(visitor, regionData, familyData, storeFiles, false); - - visitor.familyClose(regionData, familyData); + for (ColumnFamilyDescriptor cfd : htd.getColumnFamilies()) { + Object familyData = visitor.familyOpen(regionData, cfd.getName()); + monitor.rethrowException(); + StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, htd, cfd, regionFs); + List storeFiles = tracker.load(); + if (storeFiles.isEmpty()) { + LOG.debug("No files under family: {}", cfd.getNameAsString()); + continue; } + // 2.1. build the snapshot reference for the store + // iterate through all the store's files and create "references". + addReferenceFiles(visitor, regionData, familyData, storeFiles, false); + visitor.familyClose(regionData, familyData); } visitor.regionClose(regionData); } catch (IOException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java index c2087a935198..e352303f76ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -93,7 +94,8 @@ public static void setUpBeforeClass() throws Exception { @Override protected void createTable() throws IOException, InterruptedException { MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName, - SnapshotTestingUtils.getSplitKeys(), getNumReplicas(), DelayFlushCoprocessor.class.getName(), + SnapshotTestingUtils.getSplitKeys(), getNumReplicas(), + StoreFileTrackerFactory.Trackers.DEFAULT.name(), DelayFlushCoprocessor.class.getName(), FAMILY); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java index cdc41b01e87d..0695be12d0c4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java @@ -26,8 +26,6 @@ import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Test create/using/deleting snapshots from the client @@ -41,8 +39,6 @@ public class TestMobSnapshotFromClient extends TestSnapshotFromClient { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestMobSnapshotFromClient.class); - private static final Logger LOG = LoggerFactory.getLogger(TestMobSnapshotFromClient.class); - /** * Setup the config for the cluster * @throws Exception on failure @@ -60,6 +56,7 @@ protected static void setupConf(Configuration conf) { @Override protected void createTable() throws Exception { - MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, getNumReplicas(), TEST_FAM); + MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, getNumReplicas(), trackerImpl.name(), + TEST_FAM); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index 08e33ac78d05..56a48c122a14 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -23,6 +23,7 @@ import static org.junit.Assert.fail; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; @@ -32,9 +33,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException; import org.apache.hadoop.hbase.snapshot.SnapshotManifestV1; @@ -51,7 +54,10 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,7 +70,8 @@ *

* This is an end-to-end test for the snapshot utility */ -@Category({LargeTests.class, ClientTests.class}) +@RunWith(Parameterized.class) +@Category({ LargeTests.class, ClientTests.class }) public class TestSnapshotFromClient { @ClassRule @@ -82,7 +89,16 @@ public class TestSnapshotFromClient { private static final Pattern MATCH_ALL = Pattern.compile(".*"); @Rule - public TestName name = new TestName(); + public TableNameTestRule name = new TableNameTestRule(); + + @Parameter + public StoreFileTrackerFactory.Trackers trackerImpl; + + @Parameters(name = "{index}: tracker={0}") + public static List params() { + return Arrays.asList(new Object[] { StoreFileTrackerFactory.Trackers.DEFAULT }, + new Object[] { StoreFileTrackerFactory.Trackers.FILE }); + } /** * Setup the config for the cluster @@ -109,7 +125,6 @@ protected static void setupConf(Configuration conf) { conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true); conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, ConstantSizeRegionSplitPolicy.class.getName()); - } @Before @@ -119,7 +134,8 @@ public void setup() throws Exception { protected void createTable() throws Exception { TableDescriptor htd = - TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(getNumReplicas()).build(); + TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(getNumReplicas()) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, trackerImpl.name()).build(); UTIL.createTable(htd, new byte[][] { TEST_FAM }, null); } @@ -316,7 +332,7 @@ public void testOfflineTableSnapshotWithEmptyRegions() throws Exception { @Test public void testListTableSnapshots() throws Exception { Admin admin = null; - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try { admin = UTIL.getAdmin(); @@ -401,7 +417,7 @@ public void testListTableSnapshotsWithRegex() throws Exception { @Test public void testDeleteTableSnapshots() throws Exception { Admin admin = null; - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try { admin = UTIL.getAdmin(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java index 394e62d556f9..2433af42b5c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java @@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; @@ -1073,10 +1074,9 @@ private Path splitStoreFile(final HRegionFileSystem regionFs, final RegionInfo h when(mockEnv.getMasterConfiguration()).thenReturn(new Configuration()); TableDescriptors mockTblDescs = mock(TableDescriptors.class); when(mockServices.getTableDescriptors()).thenReturn(mockTblDescs); - TableDescriptor mockTblDesc = mock(TableDescriptor.class); + TableDescriptor mockTblDesc = TableDescriptorBuilder.newBuilder(hri.getTable()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build(); when(mockTblDescs.get(any())).thenReturn(mockTblDesc); - ColumnFamilyDescriptor mockCfDesc = mock(ColumnFamilyDescriptor.class); - when(mockTblDesc.getColumnFamily(any())).thenReturn(mockCfDesc); Path regionDir = regionFs.commitDaughterRegion(hri, splitFiles, mockEnv); return new Path(new Path(regionDir, family), path.getName()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java index fea4fb4ba583..7523ae8fa0ba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; @@ -45,29 +46,40 @@ public class MobSnapshotTestingUtils { /** * Create the Mob Table. */ - public static void createMobTable(final HBaseTestingUtil util, - final TableName tableName, int regionReplication, - final byte[]... families) throws IOException, InterruptedException { - createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(), - regionReplication, families); + public static void createMobTable(final HBaseTestingUtil util, final TableName tableName, + int regionReplication, final byte[]... families) throws IOException, InterruptedException { + createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(), regionReplication, + StoreFileTrackerFactory.Trackers.DEFAULT.name(), families); + } + + public static void createMobTable(final HBaseTestingUtil util, final TableName tableName, + int regionReplication, String storeFileTracker, final byte[]... families) + throws IOException, InterruptedException { + createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(), regionReplication, + storeFileTracker, families); } - public static void createPreSplitMobTable(final HBaseTestingUtil util, - final TableName tableName, int nRegions, final byte[]... families) - throws IOException, InterruptedException { - createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(nRegions), - 1, families); + public static void createPreSplitMobTable(final HBaseTestingUtil util, final TableName tableName, + int nRegions, final byte[]... families) throws IOException, InterruptedException { + createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(nRegions), 1, families); + } + + public static void createMobTable(final HBaseTestingUtil util, final TableName tableName, + final byte[][] splitKeys, int regionReplication, final byte[]... families) + throws IOException, InterruptedException { + createMobTable(util, tableName, splitKeys, regionReplication, + StoreFileTrackerFactory.Trackers.DEFAULT.name(), families); } public static void createMobTable(final HBaseTestingUtil util, final TableName tableName, - final byte[][] splitKeys, int regionReplication, final byte[]... families) - throws IOException, InterruptedException { - createMobTable(util, tableName, splitKeys, regionReplication, null, families); + final byte[][] splitKeys, int regionReplication, String storeFileTracker, + final byte[]... families) throws IOException, InterruptedException { + createMobTable(util, tableName, splitKeys, regionReplication, storeFileTracker, null, families); } - public static void createMobTable(HBaseTestingUtil util, TableName tableName, - byte[][] splitKeys, int regionReplication, String cpClassName, byte[]... families) - throws IOException, InterruptedException { + public static void createMobTable(HBaseTestingUtil util, TableName tableName, byte[][] splitKeys, + int regionReplication, String storeFileTracker, String cpClassName, byte[]... families) + throws IOException, InterruptedException { TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(regionReplication); for (byte[] family : families) { @@ -77,6 +89,7 @@ public static void createMobTable(HBaseTestingUtil util, TableName tableName, if (!StringUtils.isBlank(cpClassName)) { builder.setCoprocessor(cpClassName); } + builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, storeFileTracker); util.getAdmin().createTable(builder.build(), splitKeys); SnapshotTestingUtils.waitForTableToBeOnline(util, tableName); assertEquals((splitKeys.length + 1) * regionReplication, @@ -85,15 +98,10 @@ public static void createMobTable(HBaseTestingUtil util, TableName tableName, /** * Create a Mob table. - * - * @param util - * @param tableName - * @param families * @return An Table instance for the created table. - * @throws IOException */ - public static Table createMobTable(final HBaseTestingUtil util, - final TableName tableName, final byte[]... families) throws IOException { + public static Table createMobTable(final HBaseTestingUtil util, final TableName tableName, + final byte[]... families) throws IOException { TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); for (byte[] family : families) { // Disable blooms (they are on by default as of 0.95) but we disable them @@ -102,10 +110,7 @@ public static Table createMobTable(final HBaseTestingUtil util, // and blooms being // on is interfering. builder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family) - .setBloomFilterType(BloomType.NONE) - .setMobEnabled(true) - .setMobThreshold(0L) - .build()); + .setBloomFilterType(BloomType.NONE).setMobEnabled(true).setMobThreshold(0L).build()); } util.getAdmin().createTable(builder.build()); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait @@ -135,8 +140,8 @@ public static int countMobRows(final Table table, final byte[]... families) thro } } - public static void verifyMobRowCount(final HBaseTestingUtil util, - final TableName tableName, long expectedRows) throws IOException { + public static void verifyMobRowCount(final HBaseTestingUtil util, final TableName tableName, + long expectedRows) throws IOException { Table table = ConnectionFactory.createConnection(util.getConfiguration()).getTable(tableName); try { @@ -156,13 +161,10 @@ public SnapshotMock(final Configuration conf, final FileSystem fs, final Path ro @Override public TableDescriptor createHtd(final String tableName) { - return TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)) - .setColumnFamily(ColumnFamilyDescriptorBuilder - .newBuilder(Bytes.toBytes(TEST_FAMILY)) - .setMobEnabled(true) - .setMobThreshold(0L) - .build()) - .build(); + return TableDescriptorBuilder + .newBuilder(TableName.valueOf(tableName)).setColumnFamily(ColumnFamilyDescriptorBuilder + .newBuilder(Bytes.toBytes(TEST_FAMILY)).setMobEnabled(true).setMobThreshold(0L).build()) + .build(); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java index d8d2a5eed0b1..00d2e84a464b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java @@ -480,9 +480,8 @@ public SnapshotBuilder(final Configuration conf, final FileSystem fs, this.desc = desc; this.tableRegions = tableRegions; this.snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf); - new FSTableDescriptors(conf) - .createTableDescriptorForTableDirectory(this.snapshotDir.getFileSystem(conf), - snapshotDir, htd, false); + FSTableDescriptors.createTableDescriptorForTableDirectory( + this.snapshotDir.getFileSystem(conf), snapshotDir, htd, false); } public TableDescriptor getTableDescriptor() { @@ -502,15 +501,13 @@ public Path[] addRegion() throws IOException { } public Path[] addRegionV1() throws IOException { - return addRegion(desc.toBuilder() - .setVersion(SnapshotManifestV1.DESCRIPTOR_VERSION) - .build()); + return addRegion( + desc.toBuilder().setVersion(SnapshotManifestV1.DESCRIPTOR_VERSION).build()); } public Path[] addRegionV2() throws IOException { - return addRegion(desc.toBuilder() - .setVersion(SnapshotManifestV2.DESCRIPTOR_VERSION) - .build()); + return addRegion( + desc.toBuilder().setVersion(SnapshotManifestV2.DESCRIPTOR_VERSION).build()); } private Path[] addRegion(final SnapshotProtos.SnapshotDescription desc) throws IOException { @@ -521,6 +518,7 @@ private Path[] addRegion(final SnapshotProtos.SnapshotDescription desc) throws I RegionData regionData = tableRegions[this.snapshotted++]; ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getName()); SnapshotManifest manifest = SnapshotManifest.create(conf, fs, snapshotDir, desc, monitor); + manifest.addTableDescriptor(htd); manifest.addRegion(regionData.tableDir, regionData.hri); return regionData.files; } From 4f0a8a1894eeae77dd0d3bad3fac913a4234250d Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Wed, 13 Oct 2021 15:48:13 +0100 Subject: [PATCH 09/26] =?UTF-8?q?HBASE-26326=20CreateTableProcedure=20fail?= =?UTF-8?q?s=20when=20FileBasedStoreFileTracker=E2=80=A6=20(#3721)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Duo Zhang Signed-off-by: Josh Elser --- .../procedure/CreateTableProcedure.java | 6 ++---- .../FileBasedStoreFileTracker.java | 9 ++++++++- .../MigrationStoreFileTracker.java | 12 +---------- .../storefiletracker/StoreFileTracker.java | 5 +++-- .../StoreFileTrackerBase.java | 9 ++++----- .../StoreFileTrackerFactory.java | 20 +++++++++++++------ .../procedure/TestCreateTableProcedure.java | 16 +++++++++++++++ .../TestStoreFileTracker.java | 2 +- 8 files changed, 49 insertions(+), 30 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java index ee8e51ffe86b..0a6a469328a7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterFileSystem; @@ -290,9 +289,8 @@ private void preCreate(final MasterProcedureEnv env) (newRegions != null ? newRegions.size() : 0)); } - TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); - StoreFileTrackerFactory.persistTrackerConfig(env.getMasterConfiguration(), builder); - tableDescriptor = builder.build(); + tableDescriptor = StoreFileTrackerFactory.updateWithTrackerConfigs(env.getMasterConfiguration(), + tableDescriptor); final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); if (cpHost != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java index c370b87c1154..4da7911bdded 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java @@ -56,7 +56,14 @@ class FileBasedStoreFileTracker extends StoreFileTrackerBase { public FileBasedStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { super(conf, isPrimaryReplica, ctx); - backedFile = new StoreFileListFile(ctx); + //CreateTableProcedure needs to instantiate the configured SFT impl, in order to update table + //descriptors with the SFT impl specific configs. By the time this happens, the table has no + //regions nor stores yet, so it can't create a proper StoreContext. + if (ctx != null) { + backedFile = new StoreFileListFile(ctx); + } else { + backedFile = null; + } } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java index 1946d4bf7eef..230c1ec1b7a8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.regionserver.StoreContext; @@ -88,17 +89,6 @@ void set(List files) { "Should not call this method on " + getClass().getSimpleName()); } - @Override - public void persistConfiguration(TableDescriptorBuilder builder) { - super.persistConfiguration(builder); - if (StringUtils.isEmpty(builder.getValue(SRC_IMPL))) { - builder.setValue(SRC_IMPL, src.getTrackerName()); - } - if (StringUtils.isEmpty(builder.getValue(DST_IMPL))) { - builder.setValue(DST_IMPL, dst.getTrackerName()); - } - } - static Class getSrcTrackerClass(Configuration conf) { return StoreFileTrackerFactory.getStoreFileTrackerClassForMigration(conf, SRC_IMPL); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java index 59fe7ef52f96..fd8f7c99092d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.List; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; @@ -75,7 +76,7 @@ void replace(Collection compactedFiles, Collection StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException; /** - * Saves StoreFileTracker implementations specific configurations into the table descriptors. + * Adds StoreFileTracker implementations specific configurations into the table descriptor. *

* This is used to avoid accidentally data loss when changing the cluster level store file tracker * implementation, and also possible misconfiguration between master and region servers. @@ -83,5 +84,5 @@ void replace(Collection compactedFiles, Collection * See HBASE-26246 for more details. * @param builder The table descriptor builder for the given table. */ - void persistConfiguration(TableDescriptorBuilder builder); + TableDescriptorBuilder updateWithTrackerConfigs(TableDescriptorBuilder builder); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java index a786add49b21..edbaace40c26 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.crypto.Encryption; @@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; @@ -83,10 +83,9 @@ public final void replace(Collection compactedFiles, } @Override - public void persistConfiguration(TableDescriptorBuilder builder) { - if (StringUtils.isEmpty(builder.getValue(TRACKER_IMPL))) { - builder.setValue(TRACKER_IMPL, getTrackerName()); - } + public TableDescriptorBuilder updateWithTrackerConfigs(TableDescriptorBuilder builder) { + builder.setValue(TRACKER_IMPL, getTrackerName()); + return builder; } protected final String getTrackerName() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index b586027f8333..1c683ae3de62 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -24,8 +24,10 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreContext; + import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; @@ -158,12 +160,18 @@ static StoreFileTrackerBase createForMigration(Configuration conf, String config return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx); } - public static void persistTrackerConfig(Configuration conf, TableDescriptorBuilder builder) { - TableDescriptor tableDescriptor = builder.build(); - ColumnFamilyDescriptor cfDesc = tableDescriptor.getColumnFamilies()[0]; - StoreContext context = StoreContext.getBuilder().withColumnFamilyDescriptor(cfDesc).build(); - StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, context); - tracker.persistConfiguration(builder); + public static TableDescriptor updateWithTrackerConfigs(Configuration conf, + TableDescriptor descriptor) { + //CreateTableProcedure needs to instantiate the configured SFT impl, in order to update table + //descriptors with the SFT impl specific configs. By the time this happens, the table has no + //regions nor stores yet, so it can't create a proper StoreContext. + if (StringUtils.isEmpty(descriptor.getValue(TRACKER_IMPL))) { + StoreFileTracker tracker = + StoreFileTrackerFactory.create(conf, true, null); + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(descriptor); + return tracker.updateWithTrackerConfigs(builder).build(); + } + return descriptor; } // should not use MigrationStoreFileTracker for new family diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java index f432c8060d3d..51ea9f58248a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.regionserver.storefiletracker.TestStoreFileTracker; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -105,6 +106,21 @@ public void testCreateWithTrackImpl() throws Exception { assertEquals(trackerName, htd.getValue(TRACKER_IMPL)); } + @Test + public void testCreateWithFileBasedStoreTrackerImpl() throws Exception { + ProcedureExecutor procExec = getMasterProcedureExecutor(); + procExec.getEnvironment().getMasterConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.FILE.name()); + final TableName tableName = TableName.valueOf(name.getMethodName()); + TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1); + RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null); + long procId = ProcedureTestingUtility.submitAndWait(procExec, + new CreateTableProcedure(procExec.getEnvironment(), htd, regions)); + ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId)); + htd = getMaster().getTableDescriptors().get(tableName); + assertEquals(StoreFileTrackerFactory.Trackers.FILE.name(), htd.getValue(TRACKER_IMPL)); + } + @Test public void testCreateWithoutColumnFamily() throws Exception { final ProcedureExecutor procExec = getMasterProcedureExecutor(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java index 1dc9c4e53fc5..b30ca47772cb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java @@ -40,7 +40,7 @@ public class TestStoreFileTracker extends DefaultStoreFileTracker { public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { super(conf, isPrimaryReplica, ctx); - if (ctx.getRegionFileSystem() != null) { + if (ctx != null && ctx.getRegionFileSystem() != null) { this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString(); LOG.info("created storeId: {}", storeId); trackedFiles.computeIfAbsent(storeId, v -> new ArrayList<>()); From 201fadc95dcde5927cff6fda55aff59d572f01e1 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 21 Oct 2021 10:27:45 +0800 Subject: [PATCH 10/26] HBASE-26386 Refactor StoreFileTracker implementations to expose the set method (#3774) Signed-off-by: Wellington Chevreuil --- .../storefiletracker/DefaultStoreFileTracker.java | 2 +- .../storefiletracker/FileBasedStoreFileTracker.java | 2 +- .../storefiletracker/MigrationStoreFileTracker.java | 5 +---- .../storefiletracker/StoreFileTracker.java | 7 +++++-- .../storefiletracker/StoreFileTrackerBase.java | 10 ---------- 5 files changed, 8 insertions(+), 18 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java index a13b75b0dcc4..b1e298dbbe22 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java @@ -62,7 +62,7 @@ protected void doAddCompactionResults(Collection compactedFiles, } @Override - void set(List files) { + public void set(List files) { // NOOP } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java index 4da7911bdded..079b59ba0274 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java @@ -148,7 +148,7 @@ protected void doAddCompactionResults(Collection compactedFiles, } @Override - void set(List files) throws IOException { + public void set(List files) throws IOException { synchronized (storefiles) { storefiles.clear(); StoreFileList.Builder builder = StoreFileList.newBuilder(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java index 230c1ec1b7a8..a6648f291e43 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java @@ -21,9 +21,6 @@ import java.util.Collection; import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.yetus.audience.InterfaceAudience; @@ -84,7 +81,7 @@ protected void doAddCompactionResults(Collection compactedFiles, } @Override - void set(List files) { + public void set(List files) { throw new UnsupportedOperationException( "Should not call this method on " + getClass().getSimpleName()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java index fd8f7c99092d..f56a0dde4741 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java @@ -20,8 +20,6 @@ import java.io.IOException; import java.util.Collection; import java.util.List; - -import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; @@ -69,6 +67,11 @@ public interface StoreFileTracker { void replace(Collection compactedFiles, Collection newFiles) throws IOException; + /** + * Set the store files. + */ + void set(List files) throws IOException; + /** * Create a writer for writing new store files. * @return Writer for a new StoreFile diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java index edbaace40c26..d3c5859ba26e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -21,11 +21,9 @@ import java.io.IOException; import java.util.Collection; -import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.crypto.Encryption; @@ -184,12 +182,4 @@ public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) th protected abstract void doAddCompactionResults(Collection compactedFiles, Collection newFiles) throws IOException; - - /** - * used to mirror the store file list after loading when migration. - *

- * Do not add this method to the {@link StoreFileTracker} interface since we do not need this - * method in upper layer. - */ - abstract void set(List files) throws IOException; } From 5c8d55aff3d290e874f82e177fbf027d2c331377 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Fri, 22 Oct 2021 16:56:15 +0100 Subject: [PATCH 11/26] HBASE-26328 Clone snapshot doesn't load reference files into FILE SFT impl (#3749) Signed-off-by: Duo Zhang --- .../org/apache/hadoop/hbase/io/HFileLink.java | 63 +++++------- .../procedure/CloneSnapshotProcedure.java | 52 ++-------- .../StoreFileTrackerBase.java | 1 + .../hbase/snapshot/RestoreSnapshotHelper.java | 95 ++++++++++++++++--- ...estCloneSnapshotProcedureFileBasedSFT.java | 42 ++++++++ 5 files changed, 157 insertions(+), 96 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedureFileBasedSFT.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java index 74836ce39c6b..fbed724a207e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java @@ -201,7 +201,6 @@ public static boolean isHFileLink(final Path path) { return isHFileLink(path.getName()); } - /** * @param fileName File name to check. * @return True if the path is a HFileLink. @@ -322,10 +321,10 @@ public static String createHFileLinkName(final TableName tableName, * @param dstFamilyPath - Destination path (table/region/cf/) * @param hfileRegionInfo - Linked HFile Region Info * @param hfileName - Linked HFile name - * @return true if the file is created, otherwise the file exists. - * @throws IOException on file or parent directory creation failure + * @return the file link name. + * @throws IOException on file or parent directory creation failure. */ - public static boolean create(final Configuration conf, final FileSystem fs, + public static String create(final Configuration conf, final FileSystem fs, final Path dstFamilyPath, final RegionInfo hfileRegionInfo, final String hfileName) throws IOException { return create(conf, fs, dstFamilyPath, hfileRegionInfo, hfileName, true); @@ -343,10 +342,10 @@ public static boolean create(final Configuration conf, final FileSystem fs, * @param hfileRegionInfo - Linked HFile Region Info * @param hfileName - Linked HFile name * @param createBackRef - Whether back reference should be created. Defaults to true. - * @return true if the file is created, otherwise the file exists. - * @throws IOException on file or parent directory creation failure + * @return the file link name. + * @throws IOException on file or parent directory creation failure. */ - public static boolean create(final Configuration conf, final FileSystem fs, + public static String create(final Configuration conf, final FileSystem fs, final Path dstFamilyPath, final RegionInfo hfileRegionInfo, final String hfileName, final boolean createBackRef) throws IOException { TableName linkedTable = hfileRegionInfo.getTable(); @@ -366,17 +365,18 @@ public static boolean create(final Configuration conf, final FileSystem fs, * @param linkedTable - Linked Table Name * @param linkedRegion - Linked Region Name * @param hfileName - Linked HFile name - * @return true if the file is created, otherwise the file exists. - * @throws IOException on file or parent directory creation failure + * @return the file link name. + * @throws IOException on file or parent directory creation failure. */ - public static boolean create(final Configuration conf, final FileSystem fs, + public static String create(final Configuration conf, final FileSystem fs, final Path dstFamilyPath, final TableName linkedTable, final String linkedRegion, final String hfileName) throws IOException { return create(conf, fs, dstFamilyPath, linkedTable, linkedRegion, hfileName, true); } /** - * Create a new HFileLink + * Create a new HFileLink. In the event of link creation failure, this method throws an + * IOException, so that the calling upper laying can decide on how to proceed with this. * *

It also adds a back-reference to the hfile back-reference directory * to simplify the reference-count and the cleaning process. @@ -388,10 +388,10 @@ public static boolean create(final Configuration conf, final FileSystem fs, * @param linkedRegion - Linked Region Name * @param hfileName - Linked HFile name * @param createBackRef - Whether back reference should be created. Defaults to true. - * @return true if the file is created, otherwise the file exists. - * @throws IOException on file or parent directory creation failure + * @return the file link name. + * @throws IOException on file or parent directory creation failure. */ - public static boolean create(final Configuration conf, final FileSystem fs, + public static String create(final Configuration conf, final FileSystem fs, final Path dstFamilyPath, final TableName linkedTable, final String linkedRegion, final String hfileName, final boolean createBackRef) throws IOException { String familyName = dstFamilyPath.getName(); @@ -417,10 +417,10 @@ public static boolean create(final Configuration conf, final FileSystem fs, * @param linkedRegion - Linked Region Name * @param hfileName - Linked HFile name * @param createBackRef - Whether back reference should be created. Defaults to true. - * @return true if the file is created, otherwise the file exists. + * @return the file link name. * @throws IOException on file or parent directory creation failure */ - public static boolean create(final Configuration conf, final FileSystem fs, + public static String create(final Configuration conf, final FileSystem fs, final Path dstFamilyPath, final String familyName, final String dstTableName, final String dstRegionName, final TableName linkedTable, final String linkedRegion, final String hfileName, final boolean createBackRef) throws IOException { @@ -444,7 +444,9 @@ public static boolean create(final Configuration conf, final FileSystem fs, } try { // Create the link - return fs.createNewFile(new Path(dstFamilyPath, name)); + if (fs.createNewFile(new Path(dstFamilyPath, name))) { + return name; + } } catch (IOException e) { LOG.error("couldn't create the link=" + name + " for " + dstFamilyPath, e); // Revert the reference if the link creation failed @@ -453,25 +455,8 @@ public static boolean create(final Configuration conf, final FileSystem fs, } throw e; } - } - - /** - * Create a new HFileLink starting from a hfileLink name - * - *

It also adds a back-reference to the hfile back-reference directory - * to simplify the reference-count and the cleaning process. - * - * @param conf {@link Configuration} to read for the archive directory name - * @param fs {@link FileSystem} on which to write the HFileLink - * @param dstFamilyPath - Destination path (table/region/cf/) - * @param hfileLinkName - HFileLink name (it contains hfile-region-table) - * @return true if the file is created, otherwise the file exists. - * @throws IOException on file or parent directory creation failure - */ - public static boolean createFromHFileLink(final Configuration conf, final FileSystem fs, - final Path dstFamilyPath, final String hfileLinkName) - throws IOException { - return createFromHFileLink(conf, fs, dstFamilyPath, hfileLinkName, true); + throw new IOException("File link=" + name + " already exists under " + + dstFamilyPath + " folder."); } /** @@ -485,10 +470,10 @@ public static boolean createFromHFileLink(final Configuration conf, final FileSy * @param dstFamilyPath - Destination path (table/region/cf/) * @param hfileLinkName - HFileLink name (it contains hfile-region-table) * @param createBackRef - Whether back reference should be created. Defaults to true. - * @return true if the file is created, otherwise the file exists. - * @throws IOException on file or parent directory creation failure + * @return the file link name. + * @throws IOException on file or parent directory creation failure. */ - public static boolean createFromHFileLink(final Configuration conf, final FileSystem fs, + public static String createFromHFileLink(final Configuration conf, final FileSystem fs, final Path dstFamilyPath, final String hfileLinkName, final boolean createBackRef) throws IOException { Matcher m = LINK_NAME_PATTERN.matcher(hfileLinkName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index dae7b94134fe..31e7ec273a3b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsRegions; -import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; @@ -453,56 +452,25 @@ private List createFsLayout( List newRegions, final CreateHdfsRegions hdfsRegionHandler) throws IOException { final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); - final Path tempdir = mfs.getTempDir(); // 1. Create Table Descriptor // using a copy of descriptor, table will be created enabling first - final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableDescriptor.getTableName()); - if (CommonFSUtils.isExists(mfs.getFileSystem(), tempTableDir)) { + final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), + tableDescriptor.getTableName()); + if (CommonFSUtils.isExists(mfs.getFileSystem(), tableDir)) { // if the region dirs exist, will cause exception and unlimited retry (see HBASE-24546) - LOG.warn("temp table dir already exists on disk: {}, will be deleted.", tempTableDir); - CommonFSUtils.deleteDirectory(mfs.getFileSystem(), tempTableDir); + LOG.warn("temp table dir already exists on disk: {}, will be deleted.", tableDir); + CommonFSUtils.deleteDirectory(mfs.getFileSystem(), tableDir); } - ((FSTableDescriptors) (env.getMasterServices().getTableDescriptors())) - .createTableDescriptorForTableDirectory(tempTableDir, - TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false); + ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors())) + .createTableDescriptorForTableDirectory(tableDir, + TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false); // 2. Create Regions newRegions = hdfsRegionHandler.createHdfsRegions( - env, tempdir, tableDescriptor.getTableName(), newRegions); - - // 3. Move Table temp directory to the hbase root location - CreateTableProcedure.moveTempDirectoryToHBaseRoot(env, tableDescriptor, tempTableDir); - // Move Table temp mob directory to the hbase root location - Path tempMobTableDir = MobUtils.getMobTableDir(tempdir, tableDescriptor.getTableName()); - if (mfs.getFileSystem().exists(tempMobTableDir)) { - moveTempMobDirectoryToHBaseRoot(mfs, tableDescriptor, tempMobTableDir); - } - return newRegions; - } + env, mfs.getRootDir(), tableDescriptor.getTableName(), newRegions); - /** - * Move table temp mob directory to the hbase root location - * @param mfs The master file system - * @param tableDescriptor The table to operate on - * @param tempMobTableDir The temp mob directory of table - * @throws IOException If failed to move temp mob dir to hbase root dir - */ - private void moveTempMobDirectoryToHBaseRoot(final MasterFileSystem mfs, - final TableDescriptor tableDescriptor, final Path tempMobTableDir) throws IOException { - FileSystem fs = mfs.getFileSystem(); - final Path tableMobDir = - MobUtils.getMobTableDir(mfs.getRootDir(), tableDescriptor.getTableName()); - if (!fs.delete(tableMobDir, true) && fs.exists(tableMobDir)) { - throw new IOException("Couldn't delete mob table " + tableMobDir); - } - if (!fs.exists(tableMobDir.getParent())) { - fs.mkdirs(tableMobDir.getParent()); - } - if (!fs.rename(tempMobTableDir, tableMobDir)) { - throw new IOException("Unable to move mob table from temp=" + tempMobTableDir - + " to hbase root=" + tableMobDir); - } + return newRegions; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java index d3c5859ba26e..b6de32b09a0d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Collection; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java index bd5388ca4dde..7c75e4658305 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.snapshot; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -53,7 +55,10 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.security.access.AccessControlClient; import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; @@ -500,18 +505,28 @@ private void restoreRegion(final RegionInfo regionInfo, String tableName = tableDesc.getTableName().getNameAsString(); final String snapshotName = snapshotDesc.getName(); + Path regionPath = new Path(tableDir, regionInfo.getEncodedName()); + HRegionFileSystem regionFS = (fs.exists(regionPath)) ? + HRegionFileSystem.openRegionFromFileSystem(conf, fs, tableDir, regionInfo, false) : + HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, regionInfo); + // Restore families present in the table for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) { byte[] family = Bytes.toBytes(familyDir.getName()); + Set familyFiles = getTableRegionFamilyFiles(familyDir); List snapshotFamilyFiles = snapshotFiles.remove(familyDir.getName()); + List filesToTrack = new ArrayList<>(); if (snapshotFamilyFiles != null) { List hfilesToAdd = new ArrayList<>(); for (SnapshotRegionManifest.StoreFile storeFile: snapshotFamilyFiles) { if (familyFiles.contains(storeFile.getName())) { // HFile already present familyFiles.remove(storeFile.getName()); + //no need to restore already present files, but we need to add those to tracker + filesToTrack.add(new StoreFileInfo(conf, fs, + new Path(familyDir, storeFile.getName()), true)); } else { // HFile missing hfilesToAdd.add(storeFile); @@ -521,9 +536,11 @@ private void restoreRegion(final RegionInfo regionInfo, // Remove hfiles not present in the snapshot for (String hfileName: familyFiles) { Path hfile = new Path(familyDir, hfileName); - LOG.trace("Removing HFile=" + hfileName + " not present in snapshot=" + snapshotName+ - " from region=" + regionInfo.getEncodedName() + " table=" + tableName); - HFileArchiver.archiveStoreFile(conf, fs, regionInfo, tableDir, family, hfile); + if (!fs.getFileStatus(hfile).isDirectory()) { + LOG.trace("Removing HFile=" + hfileName + " not present in snapshot=" + + snapshotName + " from region=" + regionInfo.getEncodedName() + " table=" + tableName); + HFileArchiver.archiveStoreFile(conf, fs, regionInfo, tableDir, family, hfile); + } } // Restore Missing files @@ -531,7 +548,10 @@ private void restoreRegion(final RegionInfo regionInfo, LOG.debug("Restoring missing HFileLink " + storeFile.getName() + " of snapshot=" + snapshotName+ " to region=" + regionInfo.getEncodedName() + " table=" + tableName); - restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs); + String fileName = restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs); + //mark the reference file to be added to tracker + filesToTrack.add(new StoreFileInfo(conf, fs, + new Path(familyDir, fileName), true)); } } else { // Family doesn't exists in the snapshot @@ -540,12 +560,24 @@ private void restoreRegion(final RegionInfo regionInfo, HFileArchiver.archiveFamilyByFamilyDir(fs, conf, regionInfo, familyDir, family); fs.delete(familyDir, true); } + + StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, + StoreContext.getBuilder().withFamilyStoreDirectoryPath(familyDir). + withRegionFileSystem(regionFS).build()); + + //simply reset list of tracked files with the matching files + //and the extra one present in the snapshot + tracker.set(filesToTrack); } // Add families not present in the table for (Map.Entry> familyEntry: snapshotFiles.entrySet()) { Path familyDir = new Path(regionDir, familyEntry.getKey()); + StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, + StoreContext.getBuilder().withFamilyStoreDirectoryPath(familyDir). + withRegionFileSystem(regionFS).build()); + List files = new ArrayList<>(); if (!fs.mkdirs(familyDir)) { throw new IOException("Unable to create familyDir=" + familyDir); } @@ -553,8 +585,10 @@ private void restoreRegion(final RegionInfo regionInfo, for (SnapshotRegionManifest.StoreFile storeFile: familyEntry.getValue()) { LOG.trace("Adding HFileLink (Not present in the table) " + storeFile.getName() + " of snapshot " + snapshotName + " to table=" + tableName); - restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs); + String fileName = restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs); + files.add(new StoreFileInfo(conf, fs, new Path(familyDir, fileName), true)); } + tracker.set(files); } } @@ -627,7 +661,8 @@ private void cloneHdfsMobRegion(final Map region final RegionInfo region) throws IOException { // clone region info (change embedded tableName with the new one) Path clonedRegionPath = MobUtils.getMobRegionPath(rootDir, tableDesc.getTableName()); - cloneRegion(clonedRegionPath, region, regionManifests.get(region.getEncodedName())); + cloneRegion(MobUtils.getMobRegionInfo(tableDesc.getTableName()), + clonedRegionPath, region, regionManifests.get(region.getEncodedName())); } /** @@ -641,18 +676,44 @@ private void cloneHdfsMobRegion(final Map region * @param regionDir {@link Path} cloned dir * @param snapshotRegionInfo */ - private void cloneRegion(final Path regionDir, final RegionInfo snapshotRegionInfo, - final SnapshotRegionManifest manifest) throws IOException { + private void cloneRegion(final RegionInfo newRegionInfo, final Path regionDir, + final RegionInfo snapshotRegionInfo, final SnapshotRegionManifest manifest) + throws IOException { final String tableName = tableDesc.getTableName().getNameAsString(); final String snapshotName = snapshotDesc.getName(); for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) { Path familyDir = new Path(regionDir, familyFiles.getFamilyName().toStringUtf8()); + List clonedFiles = new ArrayList<>(); for (SnapshotRegionManifest.StoreFile storeFile: familyFiles.getStoreFilesList()) { LOG.info("Adding HFileLink " + storeFile.getName() +" from cloned region " + "in snapshot " + snapshotName + " to table=" + tableName); - restoreStoreFile(familyDir, snapshotRegionInfo, storeFile, createBackRefs); + if (MobUtils.isMobRegionInfo(newRegionInfo)) { + String mobFileName = HFileLink.createHFileLinkName(snapshotRegionInfo, + storeFile.getName()); + Path mobPath = new Path(familyDir, mobFileName); + if (fs.exists(mobPath)) { + fs.delete(mobPath, true); + } + restoreStoreFile(familyDir, snapshotRegionInfo, storeFile, createBackRefs); + } else { + String file = restoreStoreFile(familyDir, snapshotRegionInfo, storeFile, createBackRefs); + clonedFiles.add(new StoreFileInfo(conf, fs, new Path(familyDir, file), true)); + } + } + //we don't need to track files under mobdir + if (!MobUtils.isMobRegionInfo(newRegionInfo)) { + Path regionPath = new Path(tableDir, newRegionInfo.getEncodedName()); + HRegionFileSystem regionFS = (fs.exists(regionPath)) ? + HRegionFileSystem.openRegionFromFileSystem(conf, fs, tableDir, newRegionInfo, false) : + HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, newRegionInfo); + + StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, + StoreContext.getBuilder().withFamilyStoreDirectoryPath(familyDir). + withRegionFileSystem(regionFS).build()); + tracker.set(clonedFiles); } } + } /** @@ -668,7 +729,9 @@ private void cloneRegion(final Path regionDir, final RegionInfo snapshotRegionIn */ private void cloneRegion(final HRegion region, final RegionInfo snapshotRegionInfo, final SnapshotRegionManifest manifest) throws IOException { - cloneRegion(new Path(tableDir, region.getRegionInfo().getEncodedName()), snapshotRegionInfo, + cloneRegion(region.getRegionInfo(), + new Path(tableDir, region.getRegionInfo().getEncodedName()), + snapshotRegionInfo, manifest); } @@ -685,16 +748,16 @@ private void cloneRegion(final HRegion region, final RegionInfo snapshotRegionIn * @param createBackRef - Whether back reference should be created. Defaults to true. * @param storeFile store file name (can be a Reference, HFileLink or simple HFile) */ - private void restoreStoreFile(final Path familyDir, final RegionInfo regionInfo, + private String restoreStoreFile(final Path familyDir, final RegionInfo regionInfo, final SnapshotRegionManifest.StoreFile storeFile, final boolean createBackRef) throws IOException { String hfileName = storeFile.getName(); if (HFileLink.isHFileLink(hfileName)) { - HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName, createBackRef); + return HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName, createBackRef); } else if (StoreFileInfo.isReference(hfileName)) { - restoreReferenceFile(familyDir, regionInfo, storeFile); + return restoreReferenceFile(familyDir, regionInfo, storeFile); } else { - HFileLink.create(conf, fs, familyDir, regionInfo, hfileName, createBackRef); + return HFileLink.create(conf, fs, familyDir, regionInfo, hfileName, createBackRef); } } @@ -716,7 +779,7 @@ private void restoreStoreFile(final Path familyDir, final RegionInfo regionInfo, * @param regionInfo destination region info for the table * @param storeFile reference file name */ - private void restoreReferenceFile(final Path familyDir, final RegionInfo regionInfo, + private String restoreReferenceFile(final Path familyDir, final RegionInfo regionInfo, final SnapshotRegionManifest.StoreFile storeFile) throws IOException { String hfileName = storeFile.getName(); @@ -760,6 +823,7 @@ private void restoreReferenceFile(final Path familyDir, final RegionInfo regionI IOUtils.copyBytes(in, out, conf); } + // Add the daughter region to the map String regionName = Bytes.toString(regionsMap.get(regionInfo.getEncodedNameAsBytes())); if (regionName == null) { @@ -777,6 +841,7 @@ private void restoreReferenceFile(final Path familyDir, final RegionInfo regionI daughters.setSecond(regionName); } } + return outPath.getName(); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedureFileBasedSFT.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedureFileBasedSFT.java new file mode 100644 index 000000000000..f3ae1283b48a --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedureFileBasedSFT.java @@ -0,0 +1,42 @@ +/** + * 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.master.procedure; + +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.Trackers.FILE; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; + +@Category({ MasterTests.class, MediumTests.class}) +public class TestCloneSnapshotProcedureFileBasedSFT extends TestCloneSnapshotProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCloneSnapshotProcedureFileBasedSFT.class); + + @BeforeClass + public static void setupCluster() throws Exception { + UTIL.getConfiguration().set(TRACKER_IMPL, FILE.name()); + UTIL.getConfiguration().setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); + UTIL.startMiniCluster(1); + } +} From 71bd2861e4cbae35524471a47d7c83a1b8ac06f6 Mon Sep 17 00:00:00 2001 From: GeorryHuang Date: Sat, 6 Nov 2021 22:20:12 +0800 Subject: [PATCH 12/26] HBASE-26263 [Rolling Upgrading] Persist the StoreFileTracker configurations to TableDescriptor for existing tables (#3700) Signed-off-by: Duo Zhang Reviewed-by: Wellington Ramos Chevreuil --- .../apache/hadoop/hbase/master/HMaster.java | 6 + .../master/migrate/RollingUpgradeChore.java | 130 ++++++++++++++++++ .../MigrateStoreFileTrackerProcedure.java | 48 +++++++ .../migrate/TestMigrateStoreFileTracker.java | 107 ++++++++++++++ 4 files changed, 291 insertions(+) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/migrate/RollingUpgradeChore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index ad6969ba9abf..12843ddd8494 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -131,6 +131,7 @@ import org.apache.hadoop.hbase.master.http.MasterStatusServlet; import org.apache.hadoop.hbase.master.janitor.CatalogJanitor; import org.apache.hadoop.hbase.master.locking.LockManager; +import org.apache.hadoop.hbase.master.migrate.RollingUpgradeChore; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager; import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure; @@ -376,6 +377,7 @@ public class HMaster extends HBaseServerBase implements Maste private ReplicationBarrierCleaner replicationBarrierCleaner; private MobFileCleanerChore mobFileCleanerChore; private MobFileCompactionChore mobFileCompactionChore; + private RollingUpgradeChore rollingUpgradeChore; // used to synchronize the mobCompactionStates private final IdLock mobCompactionLock = new IdLock(); // save the information of mob compactions in tables. @@ -1222,6 +1224,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc LOG.debug("Balancer post startup initialization complete, took " + ( (EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds"); } + + this.rollingUpgradeChore = new RollingUpgradeChore(this); + getChoreService().scheduleChore(rollingUpgradeChore); } private void createMissingCFsInMetaDuringUpgrade( @@ -1713,6 +1718,7 @@ protected void stopChores() { shutdownChore(snapshotCleanerChore); shutdownChore(hbckChore); shutdownChore(regionsRecoveryChore); + shutdownChore(rollingUpgradeChore); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/migrate/RollingUpgradeChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/migrate/RollingUpgradeChore.java new file mode 100644 index 000000000000..3896b41f6625 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/migrate/RollingUpgradeChore.java @@ -0,0 +1,130 @@ +/* + * 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.master.migrate; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.regionserver.storefiletracker.MigrateStoreFileTrackerProcedure; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * To avoid too many migrating/upgrade threads to be submitted at the time during master + * initialization, RollingUpgradeChore handles all rolling-upgrade tasks. + * */ +@InterfaceAudience.Private +public class RollingUpgradeChore extends ScheduledChore { + + static final String ROLLING_UPGRADE_CHORE_PERIOD_SECONDS_KEY = + "hbase.master.rolling.upgrade.chore.period.secs"; + static final int DFAULT_ROLLING_UPGRADE_CHORE_PERIOD_SECONDS = 10; // 10 seconds by default + + static final String ROLLING_UPGRADE_CHORE_DELAY_SECONDS_KEY = + "hbase.master.rolling.upgrade.chore.delay.secs"; + static final long DEFAULT_ROLLING_UPGRADE_CHORE_DELAY_SECONDS = 30; // 30 seconds + + static final int CONCURRENT_PROCEDURES_COUNT = 5; + + private final static Logger LOG = LoggerFactory.getLogger(RollingUpgradeChore.class); + ProcedureExecutor procedureExecutor; + private TableDescriptors tableDescriptors; + private List processingProcs = new ArrayList<>(); + + public RollingUpgradeChore(MasterServices masterServices) { + this(masterServices.getConfiguration(), masterServices.getMasterProcedureExecutor(), + masterServices.getTableDescriptors(), masterServices); + } + + private RollingUpgradeChore(Configuration conf, + ProcedureExecutor procedureExecutor, TableDescriptors tableDescriptors, + Stoppable stopper) { + super(RollingUpgradeChore.class.getSimpleName(), stopper, conf + .getInt(ROLLING_UPGRADE_CHORE_PERIOD_SECONDS_KEY, + DFAULT_ROLLING_UPGRADE_CHORE_PERIOD_SECONDS), conf + .getLong(ROLLING_UPGRADE_CHORE_DELAY_SECONDS_KEY, + DEFAULT_ROLLING_UPGRADE_CHORE_DELAY_SECONDS), + TimeUnit.SECONDS); + this.procedureExecutor = procedureExecutor; + this.tableDescriptors = tableDescriptors; + } + + @Override + protected void chore() { + if (isCompletelyMigrateSFT(CONCURRENT_PROCEDURES_COUNT)) { + LOG.info("All Rolling-Upgrade tasks are complete, shutdown RollingUpgradeChore!"); + shutdown(); + } + } + + private boolean isCompletelyMigrateSFT(int concurrentCount){ + Iterator iter = processingProcs.iterator(); + while(iter.hasNext()){ + MigrateStoreFileTrackerProcedure proc = iter.next(); + if(procedureExecutor.isFinished(proc.getProcId())){ + iter.remove(); + } + } + // No new migration procedures will be submitted until + // all procedures executed last time are completed. + if (!processingProcs.isEmpty()) { + return false; + } + + Map migrateSFTTables; + try { + migrateSFTTables = tableDescriptors.getAll().entrySet().stream().filter(entry -> { + TableDescriptor td = entry.getValue(); + return StringUtils.isEmpty(td.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + }).limit(concurrentCount).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())); + } catch (IOException e) { + LOG.warn("Failed to migrate StoreFileTracker", e); + return false; + } + + if (migrateSFTTables.isEmpty()) { + LOG.info("There is no table to migrate StoreFileTracker!"); + return true; + } + + for (Map.Entry entry : migrateSFTTables.entrySet()) { + TableDescriptor tableDescriptor = entry.getValue(); + MigrateStoreFileTrackerProcedure proc = + new MigrateStoreFileTrackerProcedure(procedureExecutor.getEnvironment(), tableDescriptor); + procedureExecutor.submitProcedure(proc); + processingProcs.add(proc); + } + return false; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java new file mode 100644 index 000000000000..7cf3d1e8b5ac --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java @@ -0,0 +1,48 @@ +/** + * 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.storefiletracker; + +import java.util.Optional; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.procedure.ModifyTableDescriptorProcedure; +import org.apache.hadoop.hbase.procedure2.util.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Procedure for migrating StoreFileTracker information to table descriptor. + */ +@InterfaceAudience.Private +public class MigrateStoreFileTrackerProcedure extends ModifyTableDescriptorProcedure { + + public MigrateStoreFileTrackerProcedure(){} + + public MigrateStoreFileTrackerProcedure(MasterProcedureEnv env, TableDescriptor unmodified) { + super(env, unmodified); + } + + @Override + protected Optional modify(MasterProcedureEnv env, TableDescriptor current) { + if (StringUtils.isEmpty(current.getValue(StoreFileTrackerFactory.TRACKER_IMPL))) { + TableDescriptor td = + StoreFileTrackerFactory.updateWithTrackerConfigs(env.getMasterConfiguration(), current); + return Optional.of(td); + } + return Optional.empty(); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java new file mode 100644 index 000000000000..33325de9ca7d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java @@ -0,0 +1,107 @@ +/* + * + * 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.master.migrate; + +import java.io.IOException; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(MediumTests.class) +public class TestMigrateStoreFileTracker { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMigrateStoreFileTracker.class); + private final static String[] tables = new String[] { "t1", "t2", "t3", "t4", "t5", "t6" }; + private final static String famStr = "f1"; + private final static byte[] fam = Bytes.toBytes(famStr); + + private HBaseTestingUtil HTU; + private Configuration conf; + private TableDescriptor tableDescriptor; + + @Before + public void setUp() throws Exception { + conf = HBaseConfiguration.create(); + //Speed up the launch of RollingUpgradeChore + conf.setInt(RollingUpgradeChore.ROLLING_UPGRADE_CHORE_PERIOD_SECONDS_KEY, 1); + conf.setLong(RollingUpgradeChore.ROLLING_UPGRADE_CHORE_DELAY_SECONDS_KEY, 1); + HTU = new HBaseTestingUtil(conf); + HTU.startMiniCluster(); + } + + @After + public void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + @Test + public void testMigrateStoreFileTracker() throws IOException, InterruptedException { + //create tables to test + for (int i = 0; i < tables.length; i++) { + tableDescriptor = HTU.createModifyableTableDescriptor(tables[i]) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam).build()).build(); + HTU.createTable(tableDescriptor, null); + } + TableDescriptors tableDescriptors = HTU.getMiniHBaseCluster().getMaster().getTableDescriptors(); + for (int i = 0; i < tables.length; i++) { + TableDescriptor tdAfterCreated = tableDescriptors.get(TableName.valueOf(tables[i])); + //make sure that TRACKER_IMPL was set by default after tables have been created. + Assert.assertNotNull(tdAfterCreated.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + //Remove StoreFileTracker impl from tableDescriptor + TableDescriptor tdRemovedSFT = TableDescriptorBuilder.newBuilder(tdAfterCreated) + .removeValue(StoreFileTrackerFactory.TRACKER_IMPL).build(); + tableDescriptors.update(tdRemovedSFT); + } + HTU.getMiniHBaseCluster().stopMaster(0).join(); + HTU.getMiniHBaseCluster().startMaster(); + HTU.getMiniHBaseCluster().waitForActiveAndReadyMaster(30000); + //wait until all tables have been migrated + TableDescriptors tds = HTU.getMiniHBaseCluster().getMaster().getTableDescriptors(); + HTU.waitFor(30000, () -> { + try { + for (int i = 0; i < tables.length; i++) { + TableDescriptor td = tds.get(TableName.valueOf(tables[i])); + if (StringUtils.isEmpty(td.getValue(StoreFileTrackerFactory.TRACKER_IMPL))) { + return false; + } + } + return true; + } catch (IOException e) { + return false; + } + }); + } +} From eebda10344d4798f93c5af16b5672465ca8da343 Mon Sep 17 00:00:00 2001 From: BukrosSzabolcs Date: Tue, 9 Nov 2021 17:19:00 +0100 Subject: [PATCH 13/26] HBASE-26271 Cleanup the broken store files under data directory (#3786) Signed-off-by: Duo Zhang Signed-off-by: Josh Elser Signed-off-by: Wellington Ramos Chevreuil --- .../hbase/mob/DefaultMobStoreCompactor.java | 6 +- .../regionserver/AbstractMultiFileWriter.java | 6 +- .../regionserver/BrokenStoreFileCleaner.java | 202 ++++++++++++++++ .../DateTieredMultiFileWriter.java | 2 +- .../hbase/regionserver/HRegionFileSystem.java | 2 +- .../hbase/regionserver/HRegionServer.java | 27 +++ .../hadoop/hbase/regionserver/HStore.java | 6 + .../hbase/regionserver/StoreEngine.java | 21 ++ .../regionserver/StripeMultiFileWriter.java | 2 +- .../AbstractMultiOutputCompactor.java | 4 +- .../regionserver/compactions/Compactor.java | 45 +++- .../compactions/DateTieredCompactor.java | 6 +- .../compactions/DefaultCompactor.java | 9 +- .../compactions/StripeCompactor.java | 2 +- .../FileBasedStoreFileTracker.java | 2 +- .../MigrationStoreFileTracker.java | 2 +- .../storefiletracker/StoreFileTracker.java | 6 + .../StoreFileTrackerBase.java | 6 - .../hbase/snapshot/RestoreSnapshotHelper.java | 2 +- .../hbase/mob/FaultyMobStoreCompactor.java | 3 +- .../TestBrokenStoreFileCleaner.java | 225 ++++++++++++++++++ .../regionserver/TestCompactorMemLeak.java | 4 +- .../TestStoreFileTracker.java | 1 - 23 files changed, 556 insertions(+), 35 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 01fe0005f048..15f0a73a9df9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HMobStore; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; @@ -286,7 +285,6 @@ private void calculateMobLengthMap(SetMultimap mobRefs) throw * * @param fd File details * @param scanner Where to read from. - * @param writer Where to write to. * @param smallestReadPoint Smallest read point. * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint * @param throughputController The compaction throughput controller. @@ -295,7 +293,7 @@ private void calculateMobLengthMap(SetMultimap mobRefs) throw * @return Whether compaction ended; false if it was interrupted for any reason. */ @Override - protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer, + protected boolean performCompaction(FileDetails fd, InternalScanner scanner, long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController, boolean major, int numofFilesToCompact) throws IOException { long bytesWrittenProgressForLog = 0; @@ -665,7 +663,7 @@ private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId @Override - protected List commitWriter(StoreFileWriter writer, FileDetails fd, + protected List commitWriter(FileDetails fd, CompactionRequestImpl request) throws IOException { List newFiles = Lists.newArrayList(writer.getPath()); writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java index f250304952a3..82c3867c103c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java @@ -110,7 +110,11 @@ public List abortWriters() { return paths; } - protected abstract Collection writers(); + /** + * Returns all writers. This is used to prevent deleting currently writen storefiles + * during cleanup. + */ + public abstract Collection writers(); /** * Subclasses override this method to be called at the end of a successful sequence of append; all diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java new file mode 100644 index 000000000000..0c4807d8badc --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java @@ -0,0 +1,202 @@ +/** + * 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 java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This Chore, every time it runs, will clear the unsused HFiles in the data + * folder. + */ +@InterfaceAudience.Private +public class BrokenStoreFileCleaner extends ScheduledChore { + private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class); + public static final String BROKEN_STOREFILE_CLEANER_ENABLED = + "hbase.region.broken.storefilecleaner.enabled"; + public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false; + public static final String BROKEN_STOREFILE_CLEANER_TTL = + "hbase.region.broken.storefilecleaner.ttl"; + public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h + public static final String BROKEN_STOREFILE_CLEANER_DELAY = + "hbase.region.broken.storefilecleaner.delay"; + public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h + public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER = + "hbase.region.broken.storefilecleaner.delay.jitter"; + public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D; + public static final String BROKEN_STOREFILE_CLEANER_PERIOD = + "hbase.region.broken.storefilecleaner.period"; + public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h + + private HRegionServer regionServer; + private final AtomicBoolean enabled = new AtomicBoolean(true); + private long fileTtl; + + public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, + Configuration conf, HRegionServer regionServer) { + super("BrokenStoreFileCleaner", stopper, period, delay); + this.regionServer = regionServer; + setEnabled( + conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED)); + fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL); + } + + public boolean setEnabled(final boolean enabled) { + return this.enabled.getAndSet(enabled); + } + + public boolean getEnabled() { + return this.enabled.get(); + } + + @Override + public void chore() { + if (getEnabled()) { + long start = EnvironmentEdgeManager.currentTime(); + AtomicLong deletedFiles = new AtomicLong(0); + AtomicLong failedDeletes = new AtomicLong(0); + for (HRegion region : regionServer.getRegions()) { + for (HStore store : region.getStores()) { + //only do cleanup in stores not using tmp directories + if (store.getStoreEngine().requireWritingToTmpDirFirst()) { + continue; + } + Path storePath = + new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName()); + + try { + List fsStoreFiles = + Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath)); + fsStoreFiles.forEach( + file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes)); + } catch (IOException e) { + LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath); + continue; + } + } + } + LOG.debug( + "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed " + + "to delete {}", + regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start, + deletedFiles.get(), failedDeletes.get()); + } else { + LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning."); + } + } + + private void cleanFileIfNeeded(FileStatus file, HStore store, + AtomicLong deletedFiles, AtomicLong failedDeletes) { + if(file.isDirectory()){ + LOG.trace("This is a Directory {}, skip cleanup", file.getPath()); + return; + } + + if(!validate(file.getPath())){ + LOG.trace("Invalid file {}, skip cleanup", file.getPath()); + return; + } + + if(!isOldEnough(file)){ + LOG.trace("Fresh file {}, skip cleanup", file.getPath()); + return; + } + + if(isActiveStorefile(file, store)){ + LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath()); + return; + } + + // Compacted files can still have readers and are cleaned by a separate chore, so they have to + // be skipped here + if(isCompactedFile(file, store)){ + LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath()); + return; + } + + if(isCompactionResultFile(file, store)){ + LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath()); + return; + } + + deleteFile(file, store, deletedFiles, failedDeletes); + } + + private boolean isCompactionResultFile(FileStatus file, HStore store) { + return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath()); + } + + // Compacted files can still have readers and are cleaned by a separate chore, so they have to + // be skipped here + private boolean isCompactedFile(FileStatus file, HStore store) { + return store.getStoreEngine().getStoreFileManager().getCompactedfiles().stream() + .anyMatch(sf -> sf.getPath().equals(file.getPath())); + } + + private boolean isActiveStorefile(FileStatus file, HStore store) { + return store.getStoreEngine().getStoreFileManager().getStorefiles().stream() + .anyMatch(sf -> sf.getPath().equals(file.getPath())); + } + + boolean validate(Path file) { + if (HFileLink.isBackReferencesDir(file) || HFileLink.isBackReferencesDir(file.getParent())) { + return true; + } + return StoreFileInfo.validateStoreFileName(file.getName()); + } + + boolean isOldEnough(FileStatus file){ + return file.getModificationTime() + fileTtl < EnvironmentEdgeManager.currentTime(); + } + + private void deleteFile(FileStatus file, HStore store, AtomicLong deletedFiles, + AtomicLong failedDeletes) { + Path filePath = file.getPath(); + LOG.debug("Removing {} from store", filePath); + try { + boolean success = store.getFileSystem().delete(filePath, false); + if (!success) { + failedDeletes.incrementAndGet(); + LOG.warn("Attempted to delete:" + filePath + + ", but couldn't. Attempt to delete on next pass."); + } + else{ + deletedFiles.incrementAndGet(); + } + } catch (IOException e) { + e = e instanceof RemoteException ? + ((RemoteException)e).unwrapRemoteException() : e; + LOG.warn("Error while deleting: " + filePath, e); + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java index 8201cb152c01..1e10eb2db231 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java @@ -71,7 +71,7 @@ public void append(Cell cell) throws IOException { } @Override - protected Collection writers() { + public Collection writers() { return lowerBoundary2Writer.values(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index 811002561f0a..8920471a86ee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -609,7 +609,7 @@ public Path commitDaughterRegion(final RegionInfo regionInfo, List allRegi writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent); HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( env.getMasterConfiguration(), fs, getTableDir(), regionInfo, false); - insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs); + insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs); } return regionDir; } 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 4bf2d9c25f1d..02944a2905d1 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 @@ -432,6 +432,8 @@ public class HRegionServer extends HBaseServerBase */ final ServerNonceManager nonceManager; + private BrokenStoreFileCleaner brokenStoreFileCleaner; + @InterfaceAudience.Private CompactedHFilesDischarger compactedFileDischarger; @@ -1831,6 +1833,9 @@ private void startServices() throws IOException { if (this.slowLogTableOpsChore != null) { choreService.scheduleChore(slowLogTableOpsChore); } + if (this.brokenStoreFileCleaner != null) { + choreService.scheduleChore(brokenStoreFileCleaner); + } // Leases is not a Thread. Internally it runs a daemon thread. If it gets // an unhandled exception, it will just exit. @@ -1910,6 +1915,22 @@ private void initializeThreads() { this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod, onlyMetaRefresh, this, this); } + + int brokenStoreFileCleanerPeriod = conf.getInt( + BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, + BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD); + int brokenStoreFileCleanerDelay = conf.getInt( + BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, + BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY); + double brokenStoreFileCleanerDelayJitter = conf.getDouble( + BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY_JITTER, + BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER); + double jitterRate = (RandomUtils.nextDouble() - 0.5D) * brokenStoreFileCleanerDelayJitter; + long jitterValue = Math.round(brokenStoreFileCleanerDelay * jitterRate); + this.brokenStoreFileCleaner = + new BrokenStoreFileCleaner((int) (brokenStoreFileCleanerDelay + jitterValue), + brokenStoreFileCleanerPeriod, this, conf, this); + registerConfigurationObservers(); } @@ -3484,6 +3505,11 @@ protected boolean clusterMode() { return !conf.getBoolean(MASTERLESS_CONFIG_NAME, false); } + @InterfaceAudience.Private + public BrokenStoreFileCleaner getBrokenStoreFileCleaner(){ + return brokenStoreFileCleaner; + } + @Override protected void stopChores() { shutdownChore(nonceManagerChore); @@ -3494,5 +3520,6 @@ protected void stopChores() { shutdownChore(storefileRefresher); shutdownChore(fsUtilizationChore); shutdownChore(slowLogTableOpsChore); + shutdownChore(brokenStoreFileCleaner); } } 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 20bf7c6bc6dc..ba892d76c2e2 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 @@ -1156,6 +1156,12 @@ protected List doCompaction(CompactionRequestImpl cr, } } replaceStoreFiles(filesToCompact, sfs, true); + + // This step is necessary for the correctness of BrokenStoreFileCleanerChore. It lets the + // CleanerChore know that compaction is done and the file can be cleaned up if compaction + // have failed. + storeEngine.resetCompactionWriter(); + if (cr.isMajor()) { majorCompactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs()); majorCompactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java index 04867295c3ae..ddb52d10ffd5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java @@ -42,9 +42,11 @@ import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; import org.apache.hadoop.hbase.regionserver.compactions.Compactor; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -532,6 +534,25 @@ public void removeCompactedFiles(Collection compactedFiles) { } } + /** + * Whether the implementation of the used storefile tracker requires you to write to temp + * directory first, i.e, does not allow broken store files under the actual data directory. + */ + public boolean requireWritingToTmpDirFirst() { + return storeFileTracker.requireWritingToTmpDirFirst(); + } + + /** + * Resets the compaction writer when the new file is committed and used as active storefile. + * This step is necessary for the correctness of BrokenStoreFileCleanerChore. It lets the + * CleanerChore know that compaction is done and the file can be cleaned up if compaction + * have failed. Currently called in + * @see HStore#doCompaction(CompactionRequestImpl, Collection, User, long, List) + */ + public void resetCompactionWriter(){ + compactor.resetWriter(); + } + @RestrictedApi(explanation = "Should only be called in TestHStore", link = "", allowedOnPath = ".*/TestHStore.java") ReadWriteLock getLock() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java index fc0598d89ac0..a4e943ac8b04 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java @@ -58,7 +58,7 @@ public void setNoStripeMetadata() { } @Override - protected Collection writers() { + public Collection writers() { return existingWriters; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java index 533be176e7a7..19b7a98627e6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java @@ -68,7 +68,7 @@ public StoreFileWriter createWriterWithStoragePolicy(String fileStoragePolicy) } @Override - protected void abortWriter(T writer) throws IOException { + protected void abortWriter() throws IOException { FileSystem fs = store.getFileSystem(); for (Path leftoverFile : writer.abortWriters()) { try { @@ -79,5 +79,7 @@ protected void abortWriter(T writer) throws IOException { e); } } + //this step signals that the target file is no longer writen and can be cleaned up + writer = null; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 47ef0f290251..0ee7d349e4c5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -25,9 +25,12 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -37,6 +40,7 @@ import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileInfo; +import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams; import org.apache.hadoop.hbase.regionserver.HStore; @@ -92,6 +96,8 @@ public abstract class Compactor { private final boolean dropCacheMajor; private final boolean dropCacheMinor; + protected T writer = null; + //TODO: depending on Store is not good but, realistically, all compactors currently do. Compactor(Configuration conf, HStore store) { this.conf = conf; @@ -324,7 +330,6 @@ protected final List compact(final CompactionRequestImpl request, // Find the smallest read point across all the Scanners. long smallestReadPoint = getSmallestReadPoint(); - T writer = null; boolean dropCache; if (request.isMajor() || request.isAllFiles()) { dropCache = this.dropCacheMajor; @@ -348,8 +353,13 @@ protected final List compact(final CompactionRequestImpl request, smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint); cleanSeqId = true; } + if (writer != null){ + LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream() + .map(n -> n.toString()) + .collect(Collectors.joining(", ", "{ ", " }"))); + } writer = sinkFactory.createWriter(scanner, fd, dropCache, request.isMajor()); - finished = performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId, + finished = performCompaction(fd, scanner, smallestReadPoint, cleanSeqId, throughputController, request.isAllFiles(), request.getFiles().size()); if (!finished) { throw new InterruptedIOException("Aborting compaction of store " + store + " in region " @@ -369,24 +379,23 @@ protected final List compact(final CompactionRequestImpl request, Closeables.close(scanner, true); } if (!finished && writer != null) { - abortWriter(writer); + abortWriter(); } } assert finished : "We should have exited the method on all error paths"; assert writer != null : "Writer should be non-null if no error"; - return commitWriter(writer, fd, request); + return commitWriter(fd, request); } - protected abstract List commitWriter(T writer, FileDetails fd, + protected abstract List commitWriter(FileDetails fd, CompactionRequestImpl request) throws IOException; - protected abstract void abortWriter(T writer) throws IOException; + protected abstract void abortWriter() throws IOException; /** * Performs the compaction. * @param fd FileDetails of cell sink writer * @param scanner Where to read from. - * @param writer Where to write to. * @param smallestReadPoint Smallest read point. * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= * smallestReadPoint @@ -394,7 +403,7 @@ protected abstract List commitWriter(T writer, FileDetails fd, * @param numofFilesToCompact the number of files to compact * @return Whether compaction ended; false if it was interrupted for some reason. */ - protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer, + protected boolean performCompaction(FileDetails fd, InternalScanner scanner, long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController, boolean major, int numofFilesToCompact) throws IOException { assert writer instanceof ShipperListener; @@ -537,4 +546,24 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo, return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs, dropDeletesFromRow, dropDeletesToRow); } + + public List getCompactionTargets(){ + if (writer == null){ + return Collections.emptyList(); + } + synchronized (writer){ + if (writer instanceof StoreFileWriter){ + return Arrays.asList(((StoreFileWriter)writer).getPath()); + } + return ((AbstractMultiFileWriter)writer).writers().stream().map(sfw -> sfw.getPath()).collect( + Collectors.toList()); + } + } + + /** + * Reset the Writer when the new storefiles were successfully added + */ + public void resetWriter(){ + writer = null; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java index fd5433082903..43e037c5e702 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java @@ -79,8 +79,10 @@ public DateTieredMultiFileWriter createWriter(InternalScanner scanner, FileDetai } @Override - protected List commitWriter(DateTieredMultiFileWriter writer, FileDetails fd, + protected List commitWriter(FileDetails fd, CompactionRequestImpl request) throws IOException { - return writer.commitWriters(fd.maxSeqId, request.isAllFiles(), request.getFiles()); + List pathList = + writer.commitWriters(fd.maxSeqId, request.isAllFiles(), request.getFiles()); + return pathList; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java index afa2429cb6e8..ad2384a97ab8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java @@ -63,7 +63,7 @@ public List compact(final CompactionRequestImpl request, } @Override - protected List commitWriter(StoreFileWriter writer, FileDetails fd, + protected List commitWriter(FileDetails fd, CompactionRequestImpl request) throws IOException { List newFiles = Lists.newArrayList(writer.getPath()); writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); @@ -72,12 +72,19 @@ protected List commitWriter(StoreFileWriter writer, FileDetails fd, } @Override + protected void abortWriter() throws IOException { + abortWriter(writer); + } + protected void abortWriter(StoreFileWriter writer) throws IOException { Path leftoverFile = writer.getPath(); try { writer.close(); } catch (IOException e) { LOG.warn("Failed to close the writer after an unfinished compaction.", e); + } finally { + //this step signals that the target file is no longer writen and can be cleaned up + writer = null; } try { store.getFileSystem().delete(leftoverFile, false); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java index 547555e3812e..060a11b41fe6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java @@ -125,7 +125,7 @@ public StripeMultiFileWriter createWriter(InternalScanner scanner, FileDetails f } @Override - protected List commitWriter(StripeMultiFileWriter writer, FileDetails fd, + protected List commitWriter(FileDetails fd, CompactionRequestImpl request) throws IOException { List newFiles = writer.commitWriters(fd.maxSeqId, request.isMajor(), request.getFiles()); assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata."; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java index 079b59ba0274..8d9b66e53d2a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java @@ -95,7 +95,7 @@ public List load() throws IOException { } @Override - protected boolean requireWritingToTmpDirFirst() { + public boolean requireWritingToTmpDirFirst() { return false; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java index a6648f291e43..53a474d3bde7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java @@ -57,7 +57,7 @@ public List load() throws IOException { } @Override - protected boolean requireWritingToTmpDirFirst() { + public boolean requireWritingToTmpDirFirst() { // Returns true if either of the two StoreFileTracker returns true. // For example, if we want to migrate from a tracker implementation which can ignore the broken // files under data directory to a tracker implementation which can not, if we still allow diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java index f56a0dde4741..aabbe8d87494 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java @@ -88,4 +88,10 @@ void replace(Collection compactedFiles, Collection * @param builder The table descriptor builder for the given table. */ TableDescriptorBuilder updateWithTrackerConfigs(TableDescriptorBuilder builder); + + /** + * Whether the implementation of this tracker requires you to write to temp directory first, i.e, + * does not allow broken store files under the actual data directory. + */ + boolean requireWritingToTmpDirFirst(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java index b6de32b09a0d..db10f4db4c4e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java @@ -173,12 +173,6 @@ public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) th return builder.build(); } - /** - * Whether the implementation of this tracker requires you to write to temp directory first, i.e, - * does not allow broken store files under the actual data directory. - */ - protected abstract boolean requireWritingToTmpDirFirst(); - protected abstract void doAddNewStoreFiles(Collection newFiles) throws IOException; protected abstract void doAddCompactionResults(Collection compactedFiles, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java index 7c75e4658305..0f8a95fc7648 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java @@ -549,7 +549,7 @@ private void restoreRegion(final RegionInfo regionInfo, " of snapshot=" + snapshotName+ " to region=" + regionInfo.getEncodedName() + " table=" + tableName); String fileName = restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs); - //mark the reference file to be added to tracker + // mark the reference file to be added to tracker filesToTrack.add(new StoreFileInfo(conf, fs, new Path(familyDir, fileName), true)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java index 50530dad69e7..d178d564f650 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.hfile.CorruptHFileException; -import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; @@ -89,7 +88,7 @@ public FaultyMobStoreCompactor(Configuration conf, HStore store) { } @Override - protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer, + protected boolean performCompaction(FileDetails fd, InternalScanner scanner, long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController, boolean major, int numofFilesToCompact) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java new file mode 100644 index 000000000000..78755a4fe772 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java @@ -0,0 +1,225 @@ +/** + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import java.io.IOException; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MediumTests.class, RegionServerTests.class }) +public class TestBrokenStoreFileCleaner { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class); + + private final HBaseTestingUtil testUtil = new HBaseTestingUtil(); + private final static byte[] fam = Bytes.toBytes("cf_1"); + private final static byte[] qual1 = Bytes.toBytes("qf_1"); + private final static byte[] val = Bytes.toBytes("val"); + private final static String junkFileName = "409fad9a751c4e8c86d7f32581bdc156"; + TableName tableName; + + + @Before + public void setUp() throws Exception { + testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, + "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker"); + testUtil.getConfiguration() + .set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true"); + testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0"); + testUtil.getConfiguration() + .set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000"); + testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0"); + testUtil.startMiniCluster(1); + } + + @After + public void tearDown() throws Exception { + testUtil.deleteTable(tableName); + testUtil.shutdownMiniCluster(); + } + + @Test + public void testDeletingJunkFile() throws Exception { + tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile"); + createTableWithData(tableName); + + HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0); + ServerName sn = testUtil.getMiniHBaseCluster() + .getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName()); + HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn); + BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner(); + + //create junk file + HStore store = region.getStore(fam); + Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName()); + Path junkFilePath = new Path(cfPath, junkFileName); + + FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath); + junkFileOS.writeUTF("hello"); + junkFileOS.close(); + + int storeFiles = store.getStorefilesCount(); + assertTrue(storeFiles > 0); + + //verify the file exist before the chore and missing afterwards + assertTrue(store.getFileSystem().exists(junkFilePath)); + cleaner.chore(); + assertFalse(store.getFileSystem().exists(junkFilePath)); + + //verify no storefile got deleted + int currentStoreFiles = store.getStorefilesCount(); + assertEquals(currentStoreFiles, storeFiles); + + } + + @Test + public void testSkippingCompactedFiles() throws Exception { + tableName = TableName.valueOf(getClass().getSimpleName() + "testSkippningCompactedFiles"); + createTableWithData(tableName); + + HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0); + + ServerName sn = testUtil.getMiniHBaseCluster() + .getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName()); + HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn); + BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner(); + + //run major compaction to generate compaced files + region.compact(true); + + //make sure there are compacted files + HStore store = region.getStore(fam); + int compactedFiles = store.getCompactedFilesCount(); + assertTrue(compactedFiles > 0); + + cleaner.chore(); + + //verify none of the compacted files were deleted + int existingCompactedFiles = store.getCompactedFilesCount(); + assertEquals(compactedFiles, existingCompactedFiles); + + //verify adding a junk file does not break anything + Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName()); + Path junkFilePath = new Path(cfPath, junkFileName); + + FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath); + junkFileOS.writeUTF("hello"); + junkFileOS.close(); + + assertTrue(store.getFileSystem().exists(junkFilePath)); + cleaner.setEnabled(true); + cleaner.chore(); + assertFalse(store.getFileSystem().exists(junkFilePath)); + + //verify compacted files are still intact + existingCompactedFiles = store.getCompactedFilesCount(); + assertEquals(compactedFiles, existingCompactedFiles); + } + + @Test + public void testJunkFileTTL() throws Exception { + tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile"); + createTableWithData(tableName); + + HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0); + ServerName sn = testUtil.getMiniHBaseCluster() + .getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName()); + HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn); + + //create junk file + HStore store = region.getStore(fam); + Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName()); + Path junkFilePath = new Path(cfPath, junkFileName); + + FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath); + junkFileOS.writeUTF("hello"); + junkFileOS.close(); + + int storeFiles = store.getStorefilesCount(); + assertTrue(storeFiles > 0); + + //verify the file exist before the chore + assertTrue(store.getFileSystem().exists(junkFilePath)); + + //set a 5 sec ttl + rs.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "5000"); + BrokenStoreFileCleaner cleaner = new BrokenStoreFileCleaner(15000000, + 0, rs, rs.getConfiguration(), rs); + cleaner.chore(); + //file is still present after chore run + assertTrue(store.getFileSystem().exists(junkFilePath)); + Thread.sleep(5000); + cleaner.chore(); + assertFalse(store.getFileSystem().exists(junkFilePath)); + + //verify no storefile got deleted + int currentStoreFiles = store.getStorefilesCount(); + assertEquals(currentStoreFiles, storeFiles); + } + + private Table createTableWithData(TableName tableName) throws IOException { + Table table = testUtil.createTable(tableName, fam); + try { + for (int i = 1; i < 10; i++) { + Put p = new Put(Bytes.toBytes("row" + i)); + p.addColumn(fam, qual1, val); + table.put(p); + } + // flush them + testUtil.getAdmin().flush(tableName); + for (int i = 11; i < 20; i++) { + Put p = new Put(Bytes.toBytes("row" + i)); + p.addColumn(fam, qual1, val); + table.put(p); + } + // flush them + testUtil.getAdmin().flush(tableName); + for (int i = 21; i < 30; i++) { + Put p = new Put(Bytes.toBytes("row" + i)); + p.addColumn(fam, qual1, val); + table.put(p); + } + // flush them + testUtil.getAdmin().flush(tableName); + } catch (IOException e) { + table.close(); + throw e; + } + return table; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactorMemLeak.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactorMemLeak.java index e0fca1fea7c5..6a0a8baa9ded 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactorMemLeak.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactorMemLeak.java @@ -128,13 +128,13 @@ public MyCompactor(Configuration conf, HStore store) { } @Override - protected List commitWriter(StoreFileWriter writer, FileDetails fd, + protected List commitWriter(FileDetails fd, CompactionRequestImpl request) throws IOException { HFileWriterImpl writerImpl = (HFileWriterImpl) writer.writer; Cell cell = writerImpl.getLastCell(); // The cell should be backend with an KeyOnlyKeyValue. IS_LAST_CELL_ON_HEAP.set(cell instanceof KeyOnlyKeyValue); - return super.commitWriter(writer, fd, request); + return super.commitWriter(fd, request); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java index b30ca47772cb..98189729ac75 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java @@ -47,7 +47,6 @@ public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreC } else { LOG.info("ctx.getRegionFileSystem() returned null. Leaving storeId null."); } - } @Override From d24e09dbb5cb14d6830b94a758121bca0465f054 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Fri, 19 Nov 2021 12:16:29 +0000 Subject: [PATCH 14/26] =?UTF-8?q?HBASE-26454=20CreateTableProcedure=20stil?= =?UTF-8?q?l=20relies=20on=20temp=20dir=20and=20renames=E2=80=A6=20(#3845)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Duo Zhang --- .../procedure/CreateTableProcedure.java | 30 +----- .../procedure/DeleteTableProcedure.java | 95 ++++++------------- .../access/SnapshotScannerHDFSAclHelper.java | 4 +- .../hbase/master/TestMasterFileSystem.java | 29 ++---- .../procedure/TestDeleteTableProcedure.java | 66 ------------- 5 files changed, 43 insertions(+), 181 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java index 0a6a469328a7..d77b95f186d4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java @@ -23,7 +23,6 @@ import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; @@ -336,41 +335,22 @@ protected static List createFsLayout(final MasterProcedureEnv env, final TableDescriptor tableDescriptor, List newRegions, final CreateHdfsRegions hdfsRegionHandler) throws IOException { final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); - final Path tempdir = mfs.getTempDir(); // 1. Create Table Descriptor // using a copy of descriptor, table will be created enabling first - final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableDescriptor.getTableName()); + final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), + tableDescriptor.getTableName()); ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors())) - .createTableDescriptorForTableDirectory(tempTableDir, tableDescriptor, false); + .createTableDescriptorForTableDirectory( + tableDir, tableDescriptor, false); // 2. Create Regions - newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir, + newRegions = hdfsRegionHandler.createHdfsRegions(env, mfs.getRootDir(), tableDescriptor.getTableName(), newRegions); - // 3. Move Table temp directory to the hbase root location - moveTempDirectoryToHBaseRoot(env, tableDescriptor, tempTableDir); - return newRegions; } - protected static void moveTempDirectoryToHBaseRoot( - final MasterProcedureEnv env, - final TableDescriptor tableDescriptor, - final Path tempTableDir) throws IOException { - final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); - final Path tableDir = - CommonFSUtils.getTableDir(mfs.getRootDir(), tableDescriptor.getTableName()); - FileSystem fs = mfs.getFileSystem(); - if (!fs.delete(tableDir, true) && fs.exists(tableDir)) { - throw new IOException("Couldn't delete " + tableDir); - } - if (!fs.rename(tempTableDir, tableDir)) { - throw new IOException("Unable to move table from temp=" + tempTableDir + - " to hbase root=" + tableDir); - } - } - protected static List addTableToMeta(final MasterProcedureEnv env, final TableDescriptor tableDescriptor, final List regions) throws IOException { assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index 80dddc7ccda1..297efc240fca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -20,10 +20,8 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.MetaTableAccessor; @@ -277,82 +275,49 @@ protected static void deleteFromFs(final MasterProcedureEnv env, final boolean archive) throws IOException { final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); final FileSystem fs = mfs.getFileSystem(); - final Path tempdir = mfs.getTempDir(); final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName); - final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableName); if (fs.exists(tableDir)) { - // Ensure temp exists - if (!fs.exists(tempdir) && !fs.mkdirs(tempdir)) { - throw new IOException("HBase temp directory '" + tempdir + "' creation failure."); - } - - // Ensure parent exists - if (!fs.exists(tempTableDir.getParent()) && !fs.mkdirs(tempTableDir.getParent())) { - throw new IOException("HBase temp directory '" + tempdir + "' creation failure."); - } - - if (fs.exists(tempTableDir)) { - // TODO - // what's in this dir? something old? probably something manual from the user... - // let's get rid of this stuff... - FileStatus[] files = fs.listStatus(tempTableDir); - if (files != null && files.length > 0) { - List regionDirList = Arrays.stream(files) - .filter(FileStatus::isDirectory) - .map(FileStatus::getPath) - .collect(Collectors.toList()); - HFileArchiver.archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(), - tempTableDir, regionDirList); + // Archive regions from FS (temp directory) + if (archive) { + List regionDirList = regions.stream().filter(RegionReplicaUtil::isDefaultReplica) + .map(region -> + FSUtils.getRegionDirFromTableDir(tableDir, region)).collect(Collectors.toList()); + HFileArchiver + .archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(), tableDir, + regionDirList); + if (!regionDirList.isEmpty()) { + LOG.debug("Archived {} regions", tableName); } - fs.delete(tempTableDir, true); } - // Move the table in /hbase/.tmp - if (!fs.rename(tableDir, tempTableDir)) { - throw new IOException("Unable to move '" + tableDir + "' to temp '" + tempTableDir + "'"); + // Archive mob data + Path mobTableDir = + CommonFSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME), tableName); + Path regionDir = new Path(mobTableDir, MobUtils.getMobRegionInfo(tableName).getEncodedName()); + if (fs.exists(regionDir)) { + HFileArchiver.archiveRegion(fs, mfs.getRootDir(), mobTableDir, regionDir); } - } - // Archive regions from FS (temp directory) - if (archive) { - List regionDirList = regions.stream().filter(RegionReplicaUtil::isDefaultReplica) - .map(region -> FSUtils.getRegionDirFromTableDir(tempTableDir, region)) - .collect(Collectors.toList()); - HFileArchiver.archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(), tempTableDir, - regionDirList); - if (!regionDirList.isEmpty()) { - LOG.debug("Archived {} regions", tableName); + // Delete table directory from FS + if (!fs.delete(tableDir, true) && fs.exists(tableDir)) { + throw new IOException("Couldn't delete " + tableDir); } - } - // Archive mob data - Path mobTableDir = - CommonFSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME), tableName); - Path regionDir = - new Path(mobTableDir, MobUtils.getMobRegionInfo(tableName).getEncodedName()); - if (fs.exists(regionDir)) { - HFileArchiver.archiveRegion(fs, mfs.getRootDir(), mobTableDir, regionDir); - } - - // Delete table directory from FS (temp directory) - if (!fs.delete(tempTableDir, true) && fs.exists(tempTableDir)) { - throw new IOException("Couldn't delete " + tempTableDir); - } - - // Delete the table directory where the mob files are saved - if (mobTableDir != null && fs.exists(mobTableDir)) { - if (!fs.delete(mobTableDir, true)) { - throw new IOException("Couldn't delete mob dir " + mobTableDir); + // Delete the table directory where the mob files are saved + if (mobTableDir != null && fs.exists(mobTableDir)) { + if (!fs.delete(mobTableDir, true)) { + throw new IOException("Couldn't delete mob dir " + mobTableDir); + } } - } - // Delete the directory on wal filesystem - FileSystem walFs = mfs.getWALFileSystem(); - Path tableWALDir = CommonFSUtils.getWALTableDir(env.getMasterConfiguration(), tableName); - if (walFs.exists(tableWALDir) && !walFs.delete(tableWALDir, true)) { - throw new IOException("Couldn't delete table dir on wal filesystem" + tableWALDir); + // Delete the directory on wal filesystem + FileSystem walFs = mfs.getWALFileSystem(); + Path tableWALDir = CommonFSUtils.getWALTableDir(env.getMasterConfiguration(), tableName); + if (walFs.exists(tableWALDir) && !walFs.delete(tableWALDir, true)) { + throw new IOException("Couldn't delete table dir on wal filesystem" + tableWALDir); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclHelper.java index 2f00b1e7a1d2..53d9970e09df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclHelper.java @@ -474,8 +474,8 @@ List getNamespaceRootPaths(String namespace) { */ List getTableRootPaths(TableName tableName, boolean includeSnapshotPath) throws IOException { - List paths = Lists.newArrayList(pathHelper.getTmpTableDir(tableName), - pathHelper.getDataTableDir(tableName), pathHelper.getMobTableDir(tableName), + List paths = Lists.newArrayList(pathHelper.getDataTableDir(tableName), + pathHelper.getMobTableDir(tableName), pathHelper.getArchiveTableDir(tableName)); if (includeSnapshotPath) { paths.addAll(getTableSnapshotPaths(tableName)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java index 60e46052a038..f08462424a50 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java @@ -18,8 +18,7 @@ package org.apache.hadoop.hbase.master; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.Assert.assertFalse; import java.util.List; import org.apache.hadoop.fs.FileSystem; @@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; -import org.apache.hadoop.hbase.util.HFileArchiveTestingUtil; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -85,7 +83,7 @@ public void testFsUriSetProperly() throws Exception { } @Test - public void testCheckTempDir() throws Exception { + public void testCheckNoTempDir() throws Exception { final MasterFileSystem masterFileSystem = UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem(); @@ -110,28 +108,13 @@ public void testCheckTempDir() throws Exception { // disable the table so that we can manipulate the files UTIL.getAdmin().disableTable(tableName); - final Path tableDir = CommonFSUtils.getTableDir(masterFileSystem.getRootDir(), tableName); final Path tempDir = masterFileSystem.getTempDir(); - final Path tempTableDir = CommonFSUtils.getTableDir(tempDir, tableName); + final Path tempNsDir = CommonFSUtils.getNamespaceDir(tempDir, + tableName.getNamespaceAsString()); final FileSystem fs = masterFileSystem.getFileSystem(); - // move the table to the temporary directory - if (!fs.rename(tableDir, tempTableDir)) { - fail(); - } - - masterFileSystem.checkTempDir(tempDir, UTIL.getConfiguration(), fs); - - // check if the temporary directory exists and is empty - assertTrue(fs.exists(tempDir)); - assertEquals(0, fs.listStatus(tempDir).length); - - // check for the existence of the archive directory - for (HRegion region : regions) { - Path archiveDir = HFileArchiveTestingUtil.getRegionArchiveDir(UTIL.getConfiguration(), - region); - assertTrue(fs.exists(archiveDir)); - } + // checks the temporary directory does not exist + assertFalse(fs.exists(tempNsDir)); UTIL.deleteTable(tableName); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java index 1dd7dc4c6206..9367a575958b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java @@ -17,34 +17,23 @@ */ package org.apache.hadoop.hbase.master.procedure; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import java.util.ArrayList; import java.util.List; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.RegionInfo; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.CommonFSUtils; -import org.apache.hadoop.hbase.util.HFileArchiveTestingUtil; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -186,59 +175,4 @@ public void testRecoveryAndDoubleExecution() throws Exception { MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName); } - - @Test - public void testDeleteWhenTempDirIsNotEmpty() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); - final String FAM = "fam"; - final byte[][] splitKeys = new byte[][] { - Bytes.toBytes("b"), Bytes.toBytes("c"), Bytes.toBytes("d") - }; - - // create the table - MasterProcedureTestingUtility.createTable( - getMasterProcedureExecutor(), tableName, splitKeys, FAM); - - // get the current store files for the regions - List regions = UTIL.getHBaseCluster().getRegions(tableName); - // make sure we have 4 regions serving this table - assertEquals(4, regions.size()); - - // load the table - try (Table table = UTIL.getConnection().getTable(tableName)) { - UTIL.loadTable(table, Bytes.toBytes(FAM)); - } - - // disable the table so that we can manipulate the files - UTIL.getAdmin().disableTable(tableName); - - final MasterFileSystem masterFileSystem = - UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem(); - final Path tableDir = CommonFSUtils.getTableDir(masterFileSystem.getRootDir(), tableName); - final Path tempDir = masterFileSystem.getTempDir(); - final Path tempTableDir = CommonFSUtils.getTableDir(tempDir, tableName); - final FileSystem fs = masterFileSystem.getFileSystem(); - - // copy the table to the temporary directory to make sure the temp directory is not empty - if (!FileUtil.copy(fs, tableDir, fs, tempTableDir, false, UTIL.getConfiguration())) { - fail(); - } - - // delete the table - final ProcedureExecutor procExec = getMasterProcedureExecutor(); - long procId = ProcedureTestingUtility.submitAndWait(procExec, - new DeleteTableProcedure(procExec.getEnvironment(), tableName)); - ProcedureTestingUtility.assertProcNotFailed(procExec, procId); - MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName); - - // check if the temporary directory is deleted - assertFalse(fs.exists(tempTableDir)); - - // check for the existence of the archive directory - for (HRegion region : regions) { - Path archiveDir = HFileArchiveTestingUtil.getRegionArchiveDir(UTIL.getConfiguration(), - region); - assertTrue(fs.exists(archiveDir)); - } - } } From 998af5dc9eef45cf7df9614d3e06e816519ac9b1 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 29 Jul 2021 18:35:19 +0800 Subject: [PATCH 15/26] HBASE-26064 Introduce a StoreFileTracker to abstract the store file tracking logic Signed-off-by: Wellington Chevreuil --- .../java/org/apache/hadoop/hbase/regionserver/StoreEngine.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java index ddb52d10ffd5..a8e23df42572 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java @@ -464,6 +464,7 @@ public List commitStoreFiles(List files, boolean validate) thr return committedFiles; } +<<<<<<< HEAD @FunctionalInterface public interface IOExceptionRunnable { void run() throws IOException; From 3c0c21a41c3462c86efe82cf67d04cf2a1fc7bfe Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Wed, 8 Sep 2021 10:31:49 +0100 Subject: [PATCH 16/26] HBASE-26079 Use StoreFileTracker when splitting and merging (#3617) Signed-off-by: Duo Zhang --- .../regionserver/storefiletracker/TestStoreFileTracker.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java index 98189729ac75..fc54eb057537 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -28,7 +27,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From d840dd31dfe0ea8b274706502a6f4c10ea66c3a2 Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Tue, 16 Nov 2021 15:30:27 +0100 Subject: [PATCH 17/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot add a clone snapshot parameter to force a specific SFT for the new table add a check to restore snapshot that prevents an SFT change that woud break the current setup tests Change-Id: I40702103af386335ea1e98cd2f1ecc27485fbefb --- .../org/apache/hadoop/hbase/client/Admin.java | 41 +++++++- .../hbase/client/AdminOverAsyncAdmin.java | 10 +- .../hadoop/hbase/client/AsyncAdmin.java | 14 ++- .../hadoop/hbase/client/AsyncHBaseAdmin.java | 9 +- .../hbase/client/ColumnFamilyDescriptor.java | 5 + .../client/ColumnFamilyDescriptorBuilder.java | 6 ++ .../hbase/client/RawAsyncHBaseAdmin.java | 22 +++-- .../main/protobuf/server/master/Master.proto | 1 + .../apache/hadoop/hbase/master/HMaster.java | 12 +-- .../hbase/master/MasterRpcServices.java | 2 +- .../procedure/CloneSnapshotProcedure.java | 35 ++++++- .../procedure/RestoreSnapshotProcedure.java | 9 +- .../master/snapshot/SnapshotManager.java | 93 ++++++++++++++++--- .../hbase/snapshot/RestoreSnapshotHelper.java | 10 +- .../TestCloneSnapshotFromClientCustomSFT.java | 55 +++++++++++ .../master/snapshot/TestSnapshotManager.java | 54 +++++++++++ .../hbase/rsgroup/VerifyingRSGroupAdmin.java | 5 +- hbase-shell/src/main/ruby/hbase/admin.rb | 4 +- hbase-shell/src/main/ruby/hbase_constants.rb | 1 + .../ruby/shell/commands/clone_snapshot.rb | 6 +- .../hbase/thrift2/client/ThriftAdmin.java | 4 +- 21 files changed, 342 insertions(+), 56 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 48893ccbcf4f..74bf973a4356 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -1620,7 +1620,7 @@ default void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) * @throws IllegalArgumentException if the restore request is formatted incorrectly */ void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) - throws IOException, RestoreSnapshotException; + throws IOException, RestoreSnapshotException; /** * Create a new table by cloning the snapshot content. @@ -1633,7 +1633,24 @@ void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean */ default void cloneSnapshot(String snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException { - cloneSnapshot(snapshotName, tableName, false); + cloneSnapshot(snapshotName, tableName, false, null); + } + + /** + * Create a new table by cloning the snapshot content. + * @param snapshotName name of the snapshot to be cloned + * @param tableName name of the table where the snapshot will be restored + * @param restoreAcl true to clone acl into newly created table + * @param cloneSFT specify the StroreFileTracker implementation used for the table + * @throws IOException if a remote or network exception occurs + * @throws TableExistsException if table to be created already exists + * @throws RestoreSnapshotException if snapshot failed to be cloned + * @throws IllegalArgumentException if the specified table has not a valid name + */ + default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String cloneSFT) + throws IOException, TableExistsException, RestoreSnapshotException { + get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl, cloneSFT), getSyncWaitTimeout(), + TimeUnit.MILLISECONDS); } /** @@ -1680,8 +1697,24 @@ default Future cloneSnapshotAsync(String snapshotName, TableName tableName * @throws RestoreSnapshotException if snapshot failed to be cloned * @throws IllegalArgumentException if the specified table has not a valid name */ - Future cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) - throws IOException, TableExistsException, RestoreSnapshotException; + default Future cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) + throws IOException, TableExistsException, RestoreSnapshotException { + return cloneSnapshotAsync(snapshotName, tableName, restoreAcl, null); + } + + /** + * Create a new table by cloning the snapshot content. + * @param snapshotName name of the snapshot to be cloned + * @param tableName name of the table where the snapshot will be restored + * @param restoreAcl true to clone acl into newly created table + * @param cloneSFT specify the StroreFileTracker implementation used for the table + * @throws IOException if a remote or network exception occurs + * @throws TableExistsException if table to be created already exists + * @throws RestoreSnapshotException if snapshot failed to be cloned + * @throws IllegalArgumentException if the specified table has not a valid name + */ + Future cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl, + String cloneSFT) throws IOException, TableExistsException, RestoreSnapshotException; /** * Execute a distributed procedure on a cluster. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java index 08de9797a16a..879cfe91404e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java @@ -644,14 +644,14 @@ public void restoreSnapshot(String snapshotName) throws IOException, RestoreSnap @Override public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) - throws IOException, RestoreSnapshotException { + throws IOException, RestoreSnapshotException { get(admin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl)); } - @Override - public Future cloneSnapshotAsync(String snapshotName, TableName tableName, - boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException { - return admin.cloneSnapshot(snapshotName, tableName, restoreAcl); + @Override public Future cloneSnapshotAsync(String snapshotName, TableName tableName, + boolean restoreAcl, String cloneSFT) + throws IOException, TableExistsException, RestoreSnapshotException { + return admin.cloneSnapshot(snapshotName, tableName, restoreAcl, cloneSFT); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index c366d3e6a7cc..fd5a3c84eac8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -895,8 +895,20 @@ default CompletableFuture cloneSnapshot(String snapshotName, TableName tab * @param tableName name of the table where the snapshot will be restored * @param restoreAcl true to restore acl of snapshot */ + default CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, + boolean restoreAcl) { + return cloneSnapshot(snapshotName, tableName, restoreAcl, null); + } + + /** + * Create a new table by cloning the snapshot content. + * @param snapshotName name of the snapshot to be cloned + * @param tableName name of the table where the snapshot will be restored + * @param restoreAcl true to restore acl of snapshot + * @param cloneSFT specify the StroreFileTracker implementation used for the table + */ CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl); + boolean restoreAcl, String cloneSFT); /** * List completed snapshots. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index f0f564be1281..8d572ada657a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -488,14 +488,15 @@ public CompletableFuture restoreSnapshot(String snapshotName) { @Override public CompletableFuture restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, - boolean restoreAcl) { - return wrap(rawAdmin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl)); + boolean restoreAcl) { + return wrap( + rawAdmin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl)); } @Override public CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl) { - return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl)); + boolean restoreAcl, String cloneSFT) { + return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl, cloneSFT)); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java index 86d561d30f06..001d672620ea 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java @@ -195,6 +195,11 @@ public interface ColumnFamilyDescriptor { * @return A clone value. Null if no mapping for the key */ Bytes getValue(Bytes key); + /** + * @param key The key. + * @return A clone value. Null if no mapping for the key + */ + String getValue(String key); /** * @param key The key. * @return A clone value. Null if no mapping for the key diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java index 06a2aec8cbff..6d85cb439c3e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java @@ -671,6 +671,12 @@ public byte[] getValue(byte[] key) { return value == null ? null : value.get(); } + @Override + public String getValue(String key) { + Bytes rval = values.get(new Bytes(Bytes.toBytes(key))); + return rval == null ? null : Bytes.toString(rval.get(), rval.getOffset(), rval.getLength()); + } + @Override public Map getValues() { return Collections.unmodifiableMap(values); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 292c9cb19bbd..5f3f2b262a2a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -1967,7 +1967,7 @@ public CompletableFuture restoreSnapshot(String snapshotName, boolean take } else if (!exists) { // if table does not exist, then just clone snapshot into new table. completeConditionalOnFuture(future, - internalRestoreSnapshot(snapshotName, finalTableName, restoreAcl)); + internalRestoreSnapshot(snapshotName, finalTableName, restoreAcl, null)); } else { addListener(isTableDisabled(finalTableName), (disabled, err4) -> { if (err4 != null) { @@ -2003,12 +2003,12 @@ private CompletableFuture restoreSnapshot(String snapshotName, TableName t future.completeExceptionally(err); } else { // Step.2 Restore snapshot - addListener(internalRestoreSnapshot(snapshotName, tableName, restoreAcl), + addListener(internalRestoreSnapshot(snapshotName, tableName, restoreAcl, null), (void2, err2) -> { if (err2 != null) { // Step.3.a Something went wrong during the restore and try to rollback. addListener( - internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName, restoreAcl), + internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName, restoreAcl, null), (void3, err3) -> { if (err3 != null) { future.completeExceptionally(err3); @@ -2036,7 +2036,7 @@ private CompletableFuture restoreSnapshot(String snapshotName, TableName t }); return future; } else { - return internalRestoreSnapshot(snapshotName, tableName, restoreAcl); + return internalRestoreSnapshot(snapshotName, tableName, restoreAcl, null); } } @@ -2053,7 +2053,7 @@ private void completeConditionalOnFuture(CompletableFuture dependentFutur @Override public CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl) { + boolean restoreAcl, String cloneSFT) { CompletableFuture future = new CompletableFuture<>(); addListener(tableExists(tableName), (exists, err) -> { if (err != null) { @@ -2062,14 +2062,14 @@ public CompletableFuture cloneSnapshot(String snapshotName, TableName tabl future.completeExceptionally(new TableExistsException(tableName)); } else { completeConditionalOnFuture(future, - internalRestoreSnapshot(snapshotName, tableName, restoreAcl)); + internalRestoreSnapshot(snapshotName, tableName, restoreAcl, cloneSFT)); } }); return future; } private CompletableFuture internalRestoreSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl) { + boolean restoreAcl, String cloneSFT) { SnapshotProtos.SnapshotDescription snapshot = SnapshotProtos.SnapshotDescription.newBuilder() .setName(snapshotName).setTable(tableName.getNameAsString()).build(); try { @@ -2077,10 +2077,14 @@ private CompletableFuture internalRestoreSnapshot(String snapshotName, Tab } catch (IllegalArgumentException e) { return failedFuture(e); } + RestoreSnapshotRequest.Builder builder = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup()) + .setNonce(ng.newNonce()).setRestoreACL(restoreAcl); + if(cloneSFT != null){ + builder.setCloneSFT(cloneSFT); + } return waitProcedureResult(this. newMasterCaller().action((controller, stub) -> this . call(controller, stub, - RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup()) - .setNonce(ng.newNonce()).setRestoreACL(restoreAcl).build(), + builder.build(), (s, c, req, done) -> s.restoreSnapshot(c, req, done), (resp) -> resp.getProcId())) .call()); } diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto index ea09e116cc71..3a3f5b5abf05 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto @@ -462,6 +462,7 @@ message RestoreSnapshotRequest { optional uint64 nonce_group = 2 [default = 0]; optional uint64 nonce = 3 [default = 0]; optional bool restoreACL = 4 [default = false]; + optional string cloneSFT = 5; } message RestoreSnapshotResponse { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 12843ddd8494..d37428681772 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -2617,8 +2617,8 @@ public TableDescriptor get() throws IOException { } - public long restoreSnapshot(final SnapshotDescription snapshotDesc, - final long nonceGroup, final long nonce, final boolean restoreAcl) throws IOException { + public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long nonceGroup, + final long nonce, final boolean restoreAcl, final String cloneSFT) throws IOException { checkInitialized(); getSnapshotManager().checkSnapshotSupport(); @@ -2628,10 +2628,10 @@ public long restoreSnapshot(final SnapshotDescription snapshotDesc, return MasterProcedureUtil.submitProcedure( new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { - @Override - protected void run() throws IOException { - setProcId( - getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl)); + @Override protected void run() throws IOException { + setProcId( + getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl, + cloneSFT)); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 46bc8c2158eb..126b2dbe32ff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -1586,7 +1586,7 @@ public RestoreSnapshotResponse restoreSnapshot(RpcController controller, RestoreSnapshotRequest request) throws ServiceException { try { long procId = server.restoreSnapshot(request.getSnapshot(), request.getNonceGroup(), - request.getNonce(), request.getRestoreACL()); + request.getNonce(), request.getRestoreACL(), request.getCloneSFT()); return RestoreSnapshotResponse.newBuilder().setProcId(procId).build(); } catch (ForeignException e) { throw new ServiceException(e.getCause()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index 31e7ec273a3b..16a7bd97b103 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -29,6 +29,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; @@ -43,6 +45,8 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.util.StringUtils; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; @@ -71,6 +75,7 @@ public class CloneSnapshotProcedure private TableDescriptor tableDescriptor; private SnapshotDescription snapshot; private boolean restoreAcl; + private String cloneSFT; private List newRegions = null; private Map > parentsToChildrenPairMap = new HashMap<>(); @@ -94,13 +99,20 @@ public CloneSnapshotProcedure(final MasterProcedureEnv env, * @param tableDescriptor the table to operate on * @param snapshot snapshot to clone from */ + public CloneSnapshotProcedure(final MasterProcedureEnv env, + final TableDescriptor tableDescriptor, final SnapshotDescription snapshot, + final boolean restoreAcl) { + this(env, tableDescriptor, snapshot, false, null); + } + public CloneSnapshotProcedure(final MasterProcedureEnv env, final TableDescriptor tableDescriptor, final SnapshotDescription snapshot, - final boolean restoreAcl) { + final boolean restoreAcl, final String cloneSFT) { super(env); this.tableDescriptor = tableDescriptor; this.snapshot = snapshot; this.restoreAcl = restoreAcl; + this.cloneSFT = cloneSFT; getMonitorStatus(); } @@ -138,6 +150,7 @@ protected Flow executeFromState(final MasterProcedureEnv env, final CloneSnapsho setNextState(CloneSnapshotState.CLONE_SNAPSHOT_WRITE_FS_LAYOUT); break; case CLONE_SNAPSHOT_WRITE_FS_LAYOUT: + updateTableDescriptorWithSFT(); newRegions = createFilesystemLayout(env, tableDescriptor, newRegions); env.getMasterServices().getTableDescriptors().update(tableDescriptor, true); setNextState(CloneSnapshotState.CLONE_SNAPSHOT_ADD_TO_META); @@ -202,6 +215,26 @@ protected Flow executeFromState(final MasterProcedureEnv env, final CloneSnapsho return Flow.HAS_MORE_STATE; } + /** + * If a StoreFileTracker is specified we strip the TableDescriptor from previous SFT config + * and set the specified SFT on the table level + */ + private void updateTableDescriptorWithSFT() { + if (StringUtils.isEmpty(cloneSFT)){ + return; + } + + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); + builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, cloneSFT); + for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()){ + ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family); + cfBuilder.setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, null); + cfBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, null); + builder.modifyColumnFamily(cfBuilder.build()); + } + tableDescriptor = builder.build(); + } + @Override protected void rollbackState(final MasterProcedureEnv env, final CloneSnapshotState state) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java index fe3da9aaf7c9..21a48a29491a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java @@ -24,6 +24,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -46,6 +47,8 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.util.StringUtils; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; @@ -89,6 +92,7 @@ public RestoreSnapshotProcedure(final MasterProcedureEnv env, throws HBaseIOException { this(env, tableDescriptor, snapshot, false); } + /** * Constructor * @param env MasterProcedureEnv @@ -382,14 +386,15 @@ private void restoreSnapshot(final MasterProcedureEnv env) throws IOException { FileSystem fs = fileSystemManager.getFileSystem(); Path rootDir = fileSystemManager.getRootDir(); final ForeignExceptionDispatcher monitorException = new ForeignExceptionDispatcher(); + final Configuration conf = new Configuration(env.getMasterConfiguration()); LOG.info("Starting restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot)); try { Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir); SnapshotManifest manifest = SnapshotManifest.open( - env.getMasterServices().getConfiguration(), fs, snapshotDir, snapshot); + conf, fs, snapshotDir, snapshot); RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper( - env.getMasterServices().getConfiguration(), + conf, fs, manifest, modifiedTableDescriptor, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index cdacff7607aa..3ffa8fd6a12a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableState; @@ -66,6 +67,9 @@ import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.procedure2.util.StringUtils; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessChecker; @@ -82,6 +86,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException; import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.NonceKey; @@ -751,8 +756,8 @@ private boolean isSnapshotCompleted(SnapshotDescription snapshot) throws IOExcep * @throws IOException */ private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName, - final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc, - final NonceKey nonceKey, final boolean restoreAcl) throws IOException { + final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc, + final NonceKey nonceKey, final boolean restoreAcl, final String cloneSFT) throws IOException { MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); TableDescriptor htd = TableDescriptorBuilder.copy(tableName, snapshotTableDesc); org.apache.hadoop.hbase.client.SnapshotDescription snapshotPOJO = null; @@ -762,7 +767,7 @@ private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableNam } long procId; try { - procId = cloneSnapshot(snapshot, htd, nonceKey, restoreAcl); + procId = cloneSnapshot(snapshot, htd, nonceKey, restoreAcl, cloneSFT); } catch (IOException e) { LOG.error("Exception occurred while cloning the snapshot " + snapshot.getName() + " as table " + tableName.getNameAsString(), e); @@ -786,7 +791,8 @@ private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableNam * @return procId the ID of the clone snapshot procedure */ synchronized long cloneSnapshot(final SnapshotDescription snapshot, - final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl) + final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl, + final String cloneSFT) throws HBaseSnapshotException { TableName tableName = tableDescriptor.getTableName(); @@ -803,7 +809,7 @@ synchronized long cloneSnapshot(final SnapshotDescription snapshot, try { long procId = master.getMasterProcedureExecutor().submitProcedure( new CloneSnapshotProcedure(master.getMasterProcedureExecutor().getEnvironment(), - tableDescriptor, snapshot, restoreAcl), + tableDescriptor, snapshot, restoreAcl, cloneSFT), nonceKey); this.restoreTableToProcIdMap.put(tableName, procId); return procId; @@ -822,7 +828,7 @@ synchronized long cloneSnapshot(final SnapshotDescription snapshot, * @throws IOException */ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final NonceKey nonceKey, - final boolean restoreAcl) throws IOException { + final boolean restoreAcl, String cloneSFT) throws IOException { FileSystem fs = master.getMasterFileSystem().getFileSystem(); Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir); @@ -854,15 +860,75 @@ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final // Execute the restore/clone operation long procId; if (master.getTableDescriptors().exists(tableName)) { - procId = restoreSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, - restoreAcl); + procId = + restoreSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, restoreAcl); } else { procId = - cloneSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, restoreAcl); + cloneSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, restoreAcl, + cloneSFT); } return procId; } + // Makes sure restoring a snapshot does not break the current SFT setup + // follows StoreUtils.createStoreConfiguration + static void checkSFTCompatibility(TableDescriptor currentTableDesc, + TableDescriptor snapshotTableDesc, Configuration baseConf) throws RestoreSnapshotException { + //have to compare TableDescriptor.values first + String tableDefault = checkIncompatibleConfig(currentTableDesc.getValue(StoreFileTrackerFactory.TRACKER_IMPL), + snapshotTableDesc.getValue(StoreFileTrackerFactory.TRACKER_IMPL), + baseConf.get(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()), + " the Table " + currentTableDesc.getTableName().getNameAsString()); + + // have to check existing CFs + for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) { + ColumnFamilyDescriptor snapCFDesc = snapshotTableDesc.getColumnFamily(cfDesc.getName()); + // if there is no counterpart in the snapshot it will be just deleted so the config does + // not matter + if (snapCFDesc != null) { + // comparing ColumnFamilyDescriptor.conf next + String cfDefault = checkIncompatibleConfig( + cfDesc.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL), + snapCFDesc.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL), tableDefault, + " the config for column family " + cfDesc.getNameAsString()); + + // then ColumnFamilyDescriptor.values + checkIncompatibleConfig(cfDesc.getValue(StoreFileTrackerFactory.TRACKER_IMPL), + snapCFDesc.getValue(StoreFileTrackerFactory.TRACKER_IMPL), cfDefault, + " the metadata of column family " + cfDesc.getNameAsString()); + } + } + } + + // check if a config change would change the behavior + static String checkIncompatibleConfig(String currentValue, String newValue, String defaultValue, + String errorMessage) throws RestoreSnapshotException { + Boolean hasIncompatibility = false; + //if there is no current override and the snapshot has an override that does not match the + //default + if (StringUtils.isEmpty(currentValue) && !StringUtils.isEmpty(newValue) && + !defaultValue.equals(newValue)) { + hasIncompatibility = true; + } + //if there is a current override + else if (!StringUtils.isEmpty(currentValue)) { + // we can only remove the override if it matches the default + if (StringUtils.isEmpty(newValue) && !defaultValue.equals(currentValue)) { + hasIncompatibility = true; + } + // the new value have to match the current one + else if (!StringUtils.isEmpty(newValue) && !currentValue.equals(newValue)) { + hasIncompatibility = true; + } + } + if (hasIncompatibility){ + throw new RestoreSnapshotException("Restoring Snapshot is not possible because " + + errorMessage + " has incompatible configuration. Current value: " + currentValue + + " Value from snapshot: " + newValue + " Default value: " + defaultValue); + } + return StringUtils.isEmpty(newValue) ? defaultValue : newValue; + } + /** * Restore the specified snapshot. The restore will fail if the destination table has a snapshot * or restore in progress. @@ -877,9 +943,14 @@ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final */ private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName, final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc, - final NonceKey nonceKey, final boolean restoreAcl) throws IOException { + final NonceKey nonceKey, final boolean restoreAcl) + throws IOException { MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); + //have to check first if restoring the snapshot would break current SFT setup + checkSFTCompatibility(master.getTableDescriptors().get(tableName), snapshotTableDesc, + master.getConfiguration()); + if (master.getTableStateManager().isTableState( TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) { throw new UnsupportedOperationException("Table '" + @@ -921,7 +992,7 @@ private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableN * @return procId the ID of the restore snapshot procedure */ private synchronized long restoreSnapshot(final SnapshotDescription snapshot, - final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl) + final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl) throws HBaseSnapshotException { final TableName tableName = tableDescriptor.getTableName(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java index 0f8a95fc7648..ff13306988c3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hbase.snapshot; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -59,6 +57,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.security.access.AccessControlClient; import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; @@ -200,8 +199,7 @@ private RestoreMetaChanges restoreHdfsRegions(final ThreadPoolExecutor exec) thr List tableRegions = getTableRegions(); - RegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor() - .getTableName()); + RegionInfo mobRegion = MobUtils.getMobRegionInfo(tableDesc.getTableName()); if (tableRegions != null) { // restore the mob region in case if (regionNames.contains(mobRegion.getEncodedName())) { @@ -707,7 +705,9 @@ private void cloneRegion(final RegionInfo newRegionInfo, final Path regionDir, HRegionFileSystem.openRegionFromFileSystem(conf, fs, tableDir, newRegionInfo, false) : HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, newRegionInfo); - StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, + Configuration sftConf = StoreUtils.createStoreConfiguration(conf, tableDesc, + tableDesc.getColumnFamily(familyFiles.getFamilyName().toByteArray())); + StoreFileTracker tracker = StoreFileTrackerFactory.create(sftConf, true, StoreContext.getBuilder().withFamilyStoreDirectoryPath(familyDir). withRegionFileSystem(regionFS).build()); tracker.set(clonedFiles); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java new file mode 100644 index 000000000000..b584e3dd9a6b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java @@ -0,0 +1,55 @@ +/** + * 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.client; + +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ LargeTests.class, ClientTests.class }) +public class TestCloneSnapshotFromClientCustomSFT extends CloneSnapshotFromClientTestBase{ + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCloneSnapshotFromClientCustomSFT.class); + + public static final String CLONE_SFT = "FILE"; + + @Test + public void testCloneSnapshotWithCustomSFT() throws IOException, InterruptedException { + TableName clonedTableName = + TableName.valueOf(getValidMethodName() + "-" + EnvironmentEdgeManager.currentTime()); + + admin.cloneSnapshot(snapshotName1, clonedTableName, false, CLONE_SFT); + verifyRowCount(TEST_UTIL, clonedTableName, snapshot1Rows); + + TableDescriptor td = admin.getDescriptor(clonedTableName); + assertEquals(CLONE_SFT, td.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + + TEST_UTIL.deleteTable(clonedTableName); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java index 1d96536d39c8..cb1e88dfa396 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.snapshot; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -28,15 +29,22 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner; import org.apache.hadoop.hbase.procedure.ProcedureCoordinator; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.zookeeper.KeeperException; import org.junit.ClassRule; import org.junit.Rule; @@ -197,4 +205,50 @@ private boolean isSnapshotSupported(final SnapshotManager manager) { return false; } } + + @Test public void testCheckIncompatibleConfig() throws Exception { + assertThrows(RestoreSnapshotException.class, () -> { + SnapshotManager.checkIncompatibleConfig("", "a", "b", "not matching default"); + }); + + assertThrows(RestoreSnapshotException.class, () -> { + SnapshotManager.checkIncompatibleConfig("a", "", "b", "reverting overwritten default"); + }); + + assertThrows(RestoreSnapshotException.class, () -> { + SnapshotManager.checkIncompatibleConfig("a", "b", "", "missmatch"); + }); + + SnapshotManager.checkIncompatibleConfig("a", "a", "", "match"); + SnapshotManager.checkIncompatibleConfig("a", "", "a", "reverting to matching default"); + SnapshotManager.checkIncompatibleConfig("", "", "a", "using default"); + } + + @Test public void testCheckSFTCompatibility() throws Exception { + //checking default value change on different configuration levels + Configuration conf = new Configuration(); + conf.set(StoreFileTrackerFactory.TRACKER_IMPL, "B"); + + //creating a TD with only TableDescriptor level config + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf("TableX")); + builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "A"); + ColumnFamilyDescriptor cf = ColumnFamilyDescriptorBuilder.of("cf"); + builder.setColumnFamily(cf); + TableDescriptor td = builder.build(); + + //creating a TD with matching ColumnFamilyDescriptor level setting + TableDescriptorBuilder snapBuilder = + TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY")); + snapBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "A"); + ColumnFamilyDescriptorBuilder snapCFBuilder = + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")); + snapCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "A"); + snapBuilder.setColumnFamily(snapCFBuilder.build()); + TableDescriptor snapTd = snapBuilder.build(); + + // adding a cf config that matches the td config is fine even when it does not match the default + SnapshotManager.checkSFTCompatibility(td, snapTd, conf); + // removing cf level config is fine when it matches the td config + SnapshotManager.checkSFTCompatibility(snapTd, td, conf); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java index c5911f2a4a87..4e67ab65f76b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java @@ -546,8 +546,9 @@ public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, b } public Future cloneSnapshotAsync(String snapshotName, TableName tableName, - boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException { - return admin.cloneSnapshotAsync(snapshotName, tableName, restoreAcl); + boolean restoreAcl, String cloneSFT) + throws IOException, TableExistsException, RestoreSnapshotException { + return admin.cloneSnapshotAsync(snapshotName, tableName, restoreAcl, cloneSFT); } public void execProcedure(String signature, String instance, Map props) diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 0d02dbaaa1d8..59496b7b6c7e 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -1248,8 +1248,8 @@ def restore_snapshot(snapshot_name, restore_acl = false) #---------------------------------------------------------------------------------------------- # Create a new table by cloning the snapshot content - def clone_snapshot(snapshot_name, table, restore_acl = false) - @admin.cloneSnapshot(snapshot_name, TableName.valueOf(table), restore_acl) + def clone_snapshot(snapshot_name, table, restore_acl = false, clone_sft = nil) + @admin.cloneSnapshot(snapshot_name, TableName.valueOf(table), restore_acl, clone_sft) end #---------------------------------------------------------------------------------------------- diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb index 1c790072a60e..5f994c7b5ae0 100644 --- a/hbase-shell/src/main/ruby/hbase_constants.rb +++ b/hbase-shell/src/main/ruby/hbase_constants.rb @@ -40,6 +40,7 @@ module HBaseConstants CACHE = 'CACHE'.freeze CACHE_BLOCKS = 'CACHE_BLOCKS'.freeze CLASSNAME = 'CLASSNAME'.freeze + CLONE_SFT = 'CLONE_SFT'.freeze CLUSTER_KEY = 'CLUSTER_KEY'.freeze COLUMN = 'COLUMN'.freeze COLUMNS = 'COLUMNS'.freeze diff --git a/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb b/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb index abc975919d92..7c8af7d80c97 100644 --- a/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb +++ b/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb @@ -33,13 +33,17 @@ def help newly created table. hbase> clone_snapshot 'snapshotName', 'namespace:tableName', {RESTORE_ACL=>true} + +StoreFileTracker implementation used after restore can be set by the following command. + hbase> clone_snapshot 'snapshotName', 'namespace:tableName', {CLONE_SFT=>FILE} EOF end def command(snapshot_name, table, args = {}) raise(ArgumentError, 'Arguments should be a Hash') unless args.is_a?(Hash) restore_acl = args.delete(::HBaseConstants::RESTORE_ACL) || false - admin.clone_snapshot(snapshot_name, table, restore_acl) + clone_sft = args.delete(::HBaseConstants::CLONE_SFT) || nil + admin.clone_snapshot(snapshot_name, table, restore_acl, clone_sft) end def handle_exceptions(cause, *args) diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java index 2b5452528283..bfce0ff7c98c 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java @@ -869,8 +869,8 @@ public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, } @Override - public Future cloneSnapshotAsync(String snapshotName, TableName tableName, boolean cloneAcl) - throws IOException, TableExistsException, RestoreSnapshotException { + public Future cloneSnapshotAsync(String snapshotName, TableName tableName, boolean cloneAcl, + String cloneSFT) throws IOException, TableExistsException, RestoreSnapshotException { throw new NotImplementedException("cloneSnapshotAsync not supported in ThriftAdmin"); } From 76f63a442a7622f1c76ef83e5b371dae903ccac5 Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Fri, 19 Nov 2021 14:23:54 +0100 Subject: [PATCH 18/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot rename variable to customSFT add variable to clone procedure serialization Change-Id: Id940fb381cb4b405549d9e6babda300428341294 --- .../org/apache/hadoop/hbase/client/Admin.java | 10 +++++----- .../hadoop/hbase/client/AdminOverAsyncAdmin.java | 7 ++++--- .../apache/hadoop/hbase/client/AsyncAdmin.java | 4 ++-- .../hadoop/hbase/client/AsyncHBaseAdmin.java | 7 +++---- .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 10 +++++----- .../src/main/protobuf/server/master/Master.proto | 2 +- .../protobuf/server/master/MasterProcedure.proto | 3 ++- .../org/apache/hadoop/hbase/master/HMaster.java | 4 ++-- .../hadoop/hbase/master/MasterRpcServices.java | 2 +- .../master/procedure/CloneSnapshotProcedure.java | 16 +++++++++++----- .../hbase/master/snapshot/SnapshotManager.java | 15 +++++++-------- .../hbase/rsgroup/VerifyingRSGroupAdmin.java | 4 ++-- .../hadoop/hbase/thrift2/client/ThriftAdmin.java | 2 +- 13 files changed, 46 insertions(+), 40 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 74bf973a4356..e2f12477b3fa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -1641,15 +1641,15 @@ default void cloneSnapshot(String snapshotName, TableName tableName) * @param snapshotName name of the snapshot to be cloned * @param tableName name of the table where the snapshot will be restored * @param restoreAcl true to clone acl into newly created table - * @param cloneSFT specify the StroreFileTracker implementation used for the table + * @param customSFT specify the StroreFileTracker used for the table * @throws IOException if a remote or network exception occurs * @throws TableExistsException if table to be created already exists * @throws RestoreSnapshotException if snapshot failed to be cloned * @throws IllegalArgumentException if the specified table has not a valid name */ - default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String cloneSFT) + default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException { - get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl, cloneSFT), getSyncWaitTimeout(), + get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); } @@ -1707,14 +1707,14 @@ default Future cloneSnapshotAsync(String snapshotName, TableName tableName * @param snapshotName name of the snapshot to be cloned * @param tableName name of the table where the snapshot will be restored * @param restoreAcl true to clone acl into newly created table - * @param cloneSFT specify the StroreFileTracker implementation used for the table + * @param customSFT specify the StroreFileTracker used for the table * @throws IOException if a remote or network exception occurs * @throws TableExistsException if table to be created already exists * @throws RestoreSnapshotException if snapshot failed to be cloned * @throws IllegalArgumentException if the specified table has not a valid name */ Future cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl, - String cloneSFT) throws IOException, TableExistsException, RestoreSnapshotException; + String customSFT) throws IOException, TableExistsException, RestoreSnapshotException; /** * Execute a distributed procedure on a cluster. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java index 879cfe91404e..161fe3df94c4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java @@ -648,10 +648,11 @@ public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, b get(admin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl)); } - @Override public Future cloneSnapshotAsync(String snapshotName, TableName tableName, - boolean restoreAcl, String cloneSFT) + @Override + public Future cloneSnapshotAsync(String snapshotName, TableName tableName, + boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException { - return admin.cloneSnapshot(snapshotName, tableName, restoreAcl, cloneSFT); + return admin.cloneSnapshot(snapshotName, tableName, restoreAcl, customSFT); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index fd5a3c84eac8..ba54bb73f7b8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -905,10 +905,10 @@ default CompletableFuture cloneSnapshot(String snapshotName, TableName tab * @param snapshotName name of the snapshot to be cloned * @param tableName name of the table where the snapshot will be restored * @param restoreAcl true to restore acl of snapshot - * @param cloneSFT specify the StroreFileTracker implementation used for the table + * @param customSFT specify the StroreFileTracker used for the table */ CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl, String cloneSFT); + boolean restoreAcl, String customSFT); /** * List completed snapshots. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index 8d572ada657a..64a13c7022f3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -489,14 +489,13 @@ public CompletableFuture restoreSnapshot(String snapshotName) { @Override public CompletableFuture restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) { - return wrap( - rawAdmin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl)); + return wrap(rawAdmin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl)); } @Override public CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl, String cloneSFT) { - return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl, cloneSFT)); + boolean restoreAcl, String customSFT) { + return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl, customSFT)); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 5f3f2b262a2a..729a64213fd5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -2053,7 +2053,7 @@ private void completeConditionalOnFuture(CompletableFuture dependentFutur @Override public CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl, String cloneSFT) { + boolean restoreAcl, String customSFT) { CompletableFuture future = new CompletableFuture<>(); addListener(tableExists(tableName), (exists, err) -> { if (err != null) { @@ -2062,14 +2062,14 @@ public CompletableFuture cloneSnapshot(String snapshotName, TableName tabl future.completeExceptionally(new TableExistsException(tableName)); } else { completeConditionalOnFuture(future, - internalRestoreSnapshot(snapshotName, tableName, restoreAcl, cloneSFT)); + internalRestoreSnapshot(snapshotName, tableName, restoreAcl, customSFT)); } }); return future; } private CompletableFuture internalRestoreSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl, String cloneSFT) { + boolean restoreAcl, String customSFT) { SnapshotProtos.SnapshotDescription snapshot = SnapshotProtos.SnapshotDescription.newBuilder() .setName(snapshotName).setTable(tableName.getNameAsString()).build(); try { @@ -2079,8 +2079,8 @@ private CompletableFuture internalRestoreSnapshot(String snapshotName, Tab } RestoreSnapshotRequest.Builder builder = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup()) .setNonce(ng.newNonce()).setRestoreACL(restoreAcl); - if(cloneSFT != null){ - builder.setCloneSFT(cloneSFT); + if(customSFT != null){ + builder.setCustomSFT(customSFT); } return waitProcedureResult(this. newMasterCaller().action((controller, stub) -> this . call(controller, stub, diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto index 3a3f5b5abf05..c502f458e267 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto @@ -462,7 +462,7 @@ message RestoreSnapshotRequest { optional uint64 nonce_group = 2 [default = 0]; optional uint64 nonce = 3 [default = 0]; optional bool restoreACL = 4 [default = false]; - optional string cloneSFT = 5; + optional string customSFT = 5; } message RestoreSnapshotResponse { diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto index 01121699880f..ab9222e3b898 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto @@ -207,6 +207,7 @@ message CloneSnapshotStateData { required TableSchema table_schema = 3; repeated RegionInfo region_info = 4; repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 5; + optional string customSFT = 6; } enum RestoreSnapshotState { @@ -654,4 +655,4 @@ enum ModifyTableDescriptorState { message ModifyTableDescriptorStateData { required TableSchema unmodified_table_schema = 1; optional TableSchema modified_table_schema = 2; -} \ No newline at end of file +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index d37428681772..e5084ce84a43 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -2618,7 +2618,7 @@ public TableDescriptor get() throws IOException { } public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long nonceGroup, - final long nonce, final boolean restoreAcl, final String cloneSFT) throws IOException { + final long nonce, final boolean restoreAcl, final String customSFT) throws IOException { checkInitialized(); getSnapshotManager().checkSnapshotSupport(); @@ -2631,7 +2631,7 @@ public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long n @Override protected void run() throws IOException { setProcId( getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl, - cloneSFT)); + customSFT)); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 126b2dbe32ff..9be3685972fd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -1586,7 +1586,7 @@ public RestoreSnapshotResponse restoreSnapshot(RpcController controller, RestoreSnapshotRequest request) throws ServiceException { try { long procId = server.restoreSnapshot(request.getSnapshot(), request.getNonceGroup(), - request.getNonce(), request.getRestoreACL(), request.getCloneSFT()); + request.getNonce(), request.getRestoreACL(), request.getCustomSFT()); return RestoreSnapshotResponse.newBuilder().setProcId(procId).build(); } catch (ForeignException e) { throw new ServiceException(e.getCause()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index 16a7bd97b103..c060b362bfc3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -75,7 +75,7 @@ public class CloneSnapshotProcedure private TableDescriptor tableDescriptor; private SnapshotDescription snapshot; private boolean restoreAcl; - private String cloneSFT; + private String customSFT; private List newRegions = null; private Map > parentsToChildrenPairMap = new HashMap<>(); @@ -107,12 +107,12 @@ public CloneSnapshotProcedure(final MasterProcedureEnv env, public CloneSnapshotProcedure(final MasterProcedureEnv env, final TableDescriptor tableDescriptor, final SnapshotDescription snapshot, - final boolean restoreAcl, final String cloneSFT) { + final boolean restoreAcl, final String customSFT) { super(env); this.tableDescriptor = tableDescriptor; this.snapshot = snapshot; this.restoreAcl = restoreAcl; - this.cloneSFT = cloneSFT; + this.customSFT = customSFT; getMonitorStatus(); } @@ -220,12 +220,12 @@ protected Flow executeFromState(final MasterProcedureEnv env, final CloneSnapsho * and set the specified SFT on the table level */ private void updateTableDescriptorWithSFT() { - if (StringUtils.isEmpty(cloneSFT)){ + if (StringUtils.isEmpty(customSFT)){ return; } TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); - builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, cloneSFT); + builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()){ ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family); cfBuilder.setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, null); @@ -322,6 +322,9 @@ protected void serializeStateData(ProcedureStateSerializer serializer) cloneSnapshotMsg.addParentToChildRegionsPairList(parentToChildrenPair); } } + if (!StringUtils.isEmpty(customSFT)){ + cloneSnapshotMsg.setCustomSFT(customSFT); + } serializer.serialize(cloneSnapshotMsg.build()); } @@ -354,6 +357,9 @@ protected void deserializeStateData(ProcedureStateSerializer serializer) parentToChildrenPair.getChild2RegionName())); } } + if (!StringUtils.isEmpty(cloneSnapshotMsg.getCustomSFT())){ + customSFT = cloneSnapshotMsg.getCustomSFT(); + } // Make sure that the monitor status is set up getMonitorStatus(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 3ffa8fd6a12a..14f70aa7f7de 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -757,7 +757,7 @@ private boolean isSnapshotCompleted(SnapshotDescription snapshot) throws IOExcep */ private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName, final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc, - final NonceKey nonceKey, final boolean restoreAcl, final String cloneSFT) throws IOException { + final NonceKey nonceKey, final boolean restoreAcl, final String customSFT) throws IOException { MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); TableDescriptor htd = TableDescriptorBuilder.copy(tableName, snapshotTableDesc); org.apache.hadoop.hbase.client.SnapshotDescription snapshotPOJO = null; @@ -767,7 +767,7 @@ private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableNam } long procId; try { - procId = cloneSnapshot(snapshot, htd, nonceKey, restoreAcl, cloneSFT); + procId = cloneSnapshot(snapshot, htd, nonceKey, restoreAcl, customSFT); } catch (IOException e) { LOG.error("Exception occurred while cloning the snapshot " + snapshot.getName() + " as table " + tableName.getNameAsString(), e); @@ -792,7 +792,7 @@ private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableNam */ synchronized long cloneSnapshot(final SnapshotDescription snapshot, final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl, - final String cloneSFT) + final String customSFT) throws HBaseSnapshotException { TableName tableName = tableDescriptor.getTableName(); @@ -809,7 +809,7 @@ synchronized long cloneSnapshot(final SnapshotDescription snapshot, try { long procId = master.getMasterProcedureExecutor().submitProcedure( new CloneSnapshotProcedure(master.getMasterProcedureExecutor().getEnvironment(), - tableDescriptor, snapshot, restoreAcl, cloneSFT), + tableDescriptor, snapshot, restoreAcl, customSFT), nonceKey); this.restoreTableToProcIdMap.put(tableName, procId); return procId; @@ -828,7 +828,7 @@ synchronized long cloneSnapshot(final SnapshotDescription snapshot, * @throws IOException */ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final NonceKey nonceKey, - final boolean restoreAcl, String cloneSFT) throws IOException { + final boolean restoreAcl, String customSFT) throws IOException { FileSystem fs = master.getMasterFileSystem().getFileSystem(); Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir); @@ -865,7 +865,7 @@ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final } else { procId = cloneSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, restoreAcl, - cloneSFT); + customSFT); } return procId; } @@ -943,8 +943,7 @@ else if (!StringUtils.isEmpty(newValue) && !currentValue.equals(newValue)) { */ private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName, final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc, - final NonceKey nonceKey, final boolean restoreAcl) - throws IOException { + final NonceKey nonceKey, final boolean restoreAcl) throws IOException { MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); //have to check first if restoring the snapshot would break current SFT setup diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java index 4e67ab65f76b..8b1a38822f28 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java @@ -546,9 +546,9 @@ public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, b } public Future cloneSnapshotAsync(String snapshotName, TableName tableName, - boolean restoreAcl, String cloneSFT) + boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException { - return admin.cloneSnapshotAsync(snapshotName, tableName, restoreAcl, cloneSFT); + return admin.cloneSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT); } public void execProcedure(String signature, String instance, Map props) diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java index bfce0ff7c98c..71d3c9e8dad7 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java @@ -870,7 +870,7 @@ public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, @Override public Future cloneSnapshotAsync(String snapshotName, TableName tableName, boolean cloneAcl, - String cloneSFT) throws IOException, TableExistsException, RestoreSnapshotException { + String customSFT) throws IOException, TableExistsException, RestoreSnapshotException { throw new NotImplementedException("cloneSnapshotAsync not supported in ThriftAdmin"); } From 444e85217ec808266c5caa3bf3480aaebae5609f Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Thu, 2 Dec 2021 21:27:25 +0100 Subject: [PATCH 19/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot rewrite restore check adopt tests Change-Id: I34f952f2a1105a55b589ca24834cbf45bfa64c65 --- .../apache/hadoop/hbase/master/HMaster.java | 3 +- .../master/snapshot/SnapshotManager.java | 77 ++++++------------- .../StoreFileTrackerFactory.java | 2 +- .../master/snapshot/TestSnapshotManager.java | 42 +++++----- 4 files changed, 48 insertions(+), 76 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index e5084ce84a43..63bca2b19956 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -2628,7 +2628,8 @@ public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long n return MasterProcedureUtil.submitProcedure( new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { - @Override protected void run() throws IOException { + @Override + protected void run() throws IOException { setProcId( getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl, customSFT)); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 14f70aa7f7de..bb39670d2631 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.util.StringUtils; +import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.security.AccessDeniedException; @@ -874,59 +875,31 @@ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final // follows StoreUtils.createStoreConfiguration static void checkSFTCompatibility(TableDescriptor currentTableDesc, TableDescriptor snapshotTableDesc, Configuration baseConf) throws RestoreSnapshotException { - //have to compare TableDescriptor.values first - String tableDefault = checkIncompatibleConfig(currentTableDesc.getValue(StoreFileTrackerFactory.TRACKER_IMPL), - snapshotTableDesc.getValue(StoreFileTrackerFactory.TRACKER_IMPL), - baseConf.get(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()), - " the Table " + currentTableDesc.getTableName().getNameAsString()); - - // have to check existing CFs - for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) { - ColumnFamilyDescriptor snapCFDesc = snapshotTableDesc.getColumnFamily(cfDesc.getName()); - // if there is no counterpart in the snapshot it will be just deleted so the config does - // not matter - if (snapCFDesc != null) { - // comparing ColumnFamilyDescriptor.conf next - String cfDefault = checkIncompatibleConfig( - cfDesc.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL), - snapCFDesc.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL), tableDefault, - " the config for column family " + cfDesc.getNameAsString()); - - // then ColumnFamilyDescriptor.values - checkIncompatibleConfig(cfDesc.getValue(StoreFileTrackerFactory.TRACKER_IMPL), - snapCFDesc.getValue(StoreFileTrackerFactory.TRACKER_IMPL), cfDefault, - " the metadata of column family " + cfDesc.getNameAsString()); - } - } - } - // check if a config change would change the behavior - static String checkIncompatibleConfig(String currentValue, String newValue, String defaultValue, - String errorMessage) throws RestoreSnapshotException { - Boolean hasIncompatibility = false; - //if there is no current override and the snapshot has an override that does not match the - //default - if (StringUtils.isEmpty(currentValue) && !StringUtils.isEmpty(newValue) && - !defaultValue.equals(newValue)) { - hasIncompatibility = true; - } - //if there is a current override - else if (!StringUtils.isEmpty(currentValue)) { - // we can only remove the override if it matches the default - if (StringUtils.isEmpty(newValue) && !defaultValue.equals(currentValue)) { - hasIncompatibility = true; - } - // the new value have to match the current one - else if (!StringUtils.isEmpty(newValue) && !currentValue.equals(newValue)) { - hasIncompatibility = true; - } - } - if (hasIncompatibility){ - throw new RestoreSnapshotException("Restoring Snapshot is not possible because " + - errorMessage + " has incompatible configuration. Current value: " + currentValue - + " Value from snapshot: " + newValue + " Default value: " + defaultValue); - } - return StringUtils.isEmpty(newValue) ? defaultValue : newValue; + for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) { + ColumnFamilyDescriptor snapCFDesc = snapshotTableDesc.getColumnFamily(cfDesc.getName()); + // if there is no counterpart in the snapshot it will be just deleted so the config does + // not matter + if (snapCFDesc != null) { + Configuration currentCompositeConf = + StoreUtils.createStoreConfiguration(baseConf, currentTableDesc, cfDesc); + Configuration snapCompositeConf = + StoreUtils.createStoreConfiguration(baseConf, snapshotTableDesc, snapCFDesc); + Class currentSFT = + StoreFileTrackerFactory.getTrackerClass(currentCompositeConf); + Class snapSFT = + StoreFileTrackerFactory.getTrackerClass(snapCompositeConf); + + //restoration is not possible if there is an SFT mismatch + if (currentSFT != snapSFT) { + throw new RestoreSnapshotException( + "Restoring Snapshot is not possible because " + " the config for column family " + + cfDesc.getNameAsString() + " has incompatible configuration. Current SFT: " + + currentSFT + " SFT from snapshot: " + snapSFT); + } + } + } + } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index 1c683ae3de62..ab9f1b333233 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -92,7 +92,7 @@ static String getStoreFileTrackerName(Class clazz) { return name != null ? name.name() : clazz.getName(); } - private static Class getTrackerClass(Configuration conf) { + public static Class getTrackerClass(Configuration conf) { try { Trackers tracker = Trackers.valueOf(getStoreFileTrackerName(conf).toUpperCase()); return tracker.clazz; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java index cb1e88dfa396..b1486f378ae8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java @@ -206,32 +206,14 @@ private boolean isSnapshotSupported(final SnapshotManager manager) { } } - @Test public void testCheckIncompatibleConfig() throws Exception { - assertThrows(RestoreSnapshotException.class, () -> { - SnapshotManager.checkIncompatibleConfig("", "a", "b", "not matching default"); - }); - - assertThrows(RestoreSnapshotException.class, () -> { - SnapshotManager.checkIncompatibleConfig("a", "", "b", "reverting overwritten default"); - }); - - assertThrows(RestoreSnapshotException.class, () -> { - SnapshotManager.checkIncompatibleConfig("a", "b", "", "missmatch"); - }); - - SnapshotManager.checkIncompatibleConfig("a", "a", "", "match"); - SnapshotManager.checkIncompatibleConfig("a", "", "a", "reverting to matching default"); - SnapshotManager.checkIncompatibleConfig("", "", "a", "using default"); - } - @Test public void testCheckSFTCompatibility() throws Exception { //checking default value change on different configuration levels Configuration conf = new Configuration(); - conf.set(StoreFileTrackerFactory.TRACKER_IMPL, "B"); + conf.set(StoreFileTrackerFactory.TRACKER_IMPL, "DEFAULT"); //creating a TD with only TableDescriptor level config TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf("TableX")); - builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "A"); + builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); ColumnFamilyDescriptor cf = ColumnFamilyDescriptorBuilder.of("cf"); builder.setColumnFamily(cf); TableDescriptor td = builder.build(); @@ -239,10 +221,10 @@ private boolean isSnapshotSupported(final SnapshotManager manager) { //creating a TD with matching ColumnFamilyDescriptor level setting TableDescriptorBuilder snapBuilder = TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY")); - snapBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "A"); + snapBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); ColumnFamilyDescriptorBuilder snapCFBuilder = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")); - snapCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "A"); + snapCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); snapBuilder.setColumnFamily(snapCFBuilder.build()); TableDescriptor snapTd = snapBuilder.build(); @@ -250,5 +232,21 @@ private boolean isSnapshotSupported(final SnapshotManager manager) { SnapshotManager.checkSFTCompatibility(td, snapTd, conf); // removing cf level config is fine when it matches the td config SnapshotManager.checkSFTCompatibility(snapTd, td, conf); + + TableDescriptorBuilder defaultBuilder = + TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY")); + defaultBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); + ColumnFamilyDescriptorBuilder defaultCFBuilder = + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")); + defaultCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "DEFAULT"); + defaultBuilder.setColumnFamily(defaultCFBuilder.build()); + TableDescriptor defaultTd = defaultBuilder.build(); + + assertThrows(RestoreSnapshotException.class, () -> { + SnapshotManager.checkSFTCompatibility(td, defaultTd, conf); + }); + assertThrows(RestoreSnapshotException.class, () -> { + SnapshotManager.checkSFTCompatibility(snapTd, defaultTd, conf); + }); } } From cc8bfa6e1aa12eff29b0586e4ce5f5ecb212efd8 Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Thu, 2 Dec 2021 22:13:12 +0100 Subject: [PATCH 20/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot small fix Change-Id: I66f9adca10a48151ae3592f4c094d20bf629c4fb --- .../java/org/apache/hadoop/hbase/regionserver/StoreEngine.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java index a8e23df42572..ddb52d10ffd5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java @@ -464,7 +464,6 @@ public List commitStoreFiles(List files, boolean validate) thr return committedFiles; } -<<<<<<< HEAD @FunctionalInterface public interface IOExceptionRunnable { void run() throws IOException; From b1bd8aa8b743d43d1561d0d5598b84adf54cca6f Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Tue, 7 Dec 2021 13:58:19 +0100 Subject: [PATCH 21/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot fix mob issue fix checkstyle --- .../org/apache/hadoop/hbase/client/Admin.java | 6 ++- .../hbase/client/RawAsyncHBaseAdmin.java | 8 +-- .../apache/hadoop/hbase/master/HMaster.java | 24 ++++----- .../procedure/CloneSnapshotProcedure.java | 2 +- .../procedure/RestoreSnapshotProcedure.java | 2 - .../master/snapshot/SnapshotManager.java | 49 +++++++++---------- .../hbase/snapshot/RestoreSnapshotHelper.java | 7 ++- 7 files changed, 48 insertions(+), 50 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index e2f12477b3fa..f80e15273df5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -1647,7 +1647,8 @@ default void cloneSnapshot(String snapshotName, TableName tableName) * @throws RestoreSnapshotException if snapshot failed to be cloned * @throws IllegalArgumentException if the specified table has not a valid name */ - default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) + default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, + String customSFT) throws IOException, TableExistsException, RestoreSnapshotException { get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); @@ -1697,7 +1698,8 @@ default Future cloneSnapshotAsync(String snapshotName, TableName tableName * @throws RestoreSnapshotException if snapshot failed to be cloned * @throws IllegalArgumentException if the specified table has not a valid name */ - default Future cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) + default Future cloneSnapshotAsync(String snapshotName, TableName tableName, + boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException { return cloneSnapshotAsync(snapshotName, tableName, restoreAcl, null); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 729a64213fd5..f0895a07f20a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -2008,7 +2008,8 @@ private CompletableFuture restoreSnapshot(String snapshotName, TableName t if (err2 != null) { // Step.3.a Something went wrong during the restore and try to rollback. addListener( - internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName, restoreAcl, null), + internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName, restoreAcl, + null), (void3, err3) -> { if (err3 != null) { future.completeExceptionally(err3); @@ -2077,8 +2078,9 @@ private CompletableFuture internalRestoreSnapshot(String snapshotName, Tab } catch (IllegalArgumentException e) { return failedFuture(e); } - RestoreSnapshotRequest.Builder builder = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup()) - .setNonce(ng.newNonce()).setRestoreACL(restoreAcl); + RestoreSnapshotRequest.Builder builder = + RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup()) + .setNonce(ng.newNonce()).setRestoreACL(restoreAcl); if(customSFT != null){ builder.setCustomSFT(customSFT); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 63bca2b19956..21bcfafd45ee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -2627,19 +2627,19 @@ public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long n getClusterSchema().getNamespace(dstTable.getNamespaceAsString()); return MasterProcedureUtil.submitProcedure( - new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { - @Override - protected void run() throws IOException { - setProcId( - getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl, - customSFT)); - } + new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + @Override + protected void run() throws IOException { + setProcId( + getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl, + customSFT)); + } - @Override - protected String getDescription() { - return "RestoreSnapshotProcedure"; - } - }); + @Override + protected String getDescription() { + return "RestoreSnapshotProcedure"; + } + }); } private void checkTableExists(final TableName tableName) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index c060b362bfc3..e720efbdff8b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -102,7 +102,7 @@ public CloneSnapshotProcedure(final MasterProcedureEnv env, public CloneSnapshotProcedure(final MasterProcedureEnv env, final TableDescriptor tableDescriptor, final SnapshotDescription snapshot, final boolean restoreAcl) { - this(env, tableDescriptor, snapshot, false, null); + this(env, tableDescriptor, snapshot, restoreAcl, null); } public CloneSnapshotProcedure(final MasterProcedureEnv env, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java index 21a48a29491a..404141450586 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java @@ -47,8 +47,6 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; -import org.apache.hadoop.hbase.procedure2.util.StringUtils; -import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index bb39670d2631..2e231c3d162b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -67,7 +67,6 @@ import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; -import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; @@ -87,7 +86,6 @@ import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException; import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.NonceKey; @@ -876,30 +874,29 @@ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final static void checkSFTCompatibility(TableDescriptor currentTableDesc, TableDescriptor snapshotTableDesc, Configuration baseConf) throws RestoreSnapshotException { - for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) { - ColumnFamilyDescriptor snapCFDesc = snapshotTableDesc.getColumnFamily(cfDesc.getName()); - // if there is no counterpart in the snapshot it will be just deleted so the config does - // not matter - if (snapCFDesc != null) { - Configuration currentCompositeConf = - StoreUtils.createStoreConfiguration(baseConf, currentTableDesc, cfDesc); - Configuration snapCompositeConf = - StoreUtils.createStoreConfiguration(baseConf, snapshotTableDesc, snapCFDesc); - Class currentSFT = - StoreFileTrackerFactory.getTrackerClass(currentCompositeConf); - Class snapSFT = - StoreFileTrackerFactory.getTrackerClass(snapCompositeConf); - - //restoration is not possible if there is an SFT mismatch - if (currentSFT != snapSFT) { - throw new RestoreSnapshotException( - "Restoring Snapshot is not possible because " + " the config for column family " - + cfDesc.getNameAsString() + " has incompatible configuration. Current SFT: " - + currentSFT + " SFT from snapshot: " + snapSFT); - } - } - } - + for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) { + ColumnFamilyDescriptor snapCFDesc = snapshotTableDesc.getColumnFamily(cfDesc.getName()); + // if there is no counterpart in the snapshot it will be just deleted so the config does + // not matter + if (snapCFDesc != null) { + Configuration currentCompositeConf = + StoreUtils.createStoreConfiguration(baseConf, currentTableDesc, cfDesc); + Configuration snapCompositeConf = + StoreUtils.createStoreConfiguration(baseConf, snapshotTableDesc, snapCFDesc); + Class currentSFT = + StoreFileTrackerFactory.getTrackerClass(currentCompositeConf); + Class snapSFT = + StoreFileTrackerFactory.getTrackerClass(snapCompositeConf); + + //restoration is not possible if there is an SFT mismatch + if (currentSFT != snapSFT) { + throw new RestoreSnapshotException( + "Restoring Snapshot is not possible because " + " the config for column family " + + cfDesc.getNameAsString() + " has incompatible configuration. Current SFT: " + + currentSFT + " SFT from snapshot: " + snapSFT); + } + } + } } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java index ff13306988c3..e4928e8fa777 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java @@ -55,9 +55,9 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; -import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.security.access.AccessControlClient; import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; @@ -71,9 +71,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; @@ -199,7 +197,8 @@ private RestoreMetaChanges restoreHdfsRegions(final ThreadPoolExecutor exec) thr List tableRegions = getTableRegions(); - RegionInfo mobRegion = MobUtils.getMobRegionInfo(tableDesc.getTableName()); + RegionInfo mobRegion = + MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor().getTableName()); if (tableRegions != null) { // restore the mob region in case if (regionNames.contains(mobRegion.getEncodedName())) { From e1f22a9d13f9be203edd29ef8d53118aaada69d9 Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Tue, 7 Dec 2021 14:09:39 +0100 Subject: [PATCH 22/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot odify the CloneSnapshotStateData to resolve conflict with HBASE-26462 --- .../src/main/protobuf/server/master/MasterProcedure.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto index ab9222e3b898..594af83053e7 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto @@ -207,7 +207,7 @@ message CloneSnapshotStateData { required TableSchema table_schema = 3; repeated RegionInfo region_info = 4; repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 5; - optional string customSFT = 6; + optional string customSFT = 7; } enum RestoreSnapshotState { From 13c5ee66911b737edd06c21c92d228972ef8c83a Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Fri, 10 Dec 2021 17:05:34 +0100 Subject: [PATCH 23/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot validate custom sft before using to clone a snapshot --- .../org/apache/hadoop/hbase/client/Admin.java | 4 +++- .../master/procedure/CloneSnapshotProcedure.java | 14 ++++++++++++++ .../TestCloneSnapshotFromClientCustomSFT.java | 16 ++++++++++++++++ .../main/ruby/shell/commands/clone_snapshot.rb | 2 +- 4 files changed, 34 insertions(+), 2 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index f80e15273df5..afe540c03422 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -69,6 +69,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.yetus.audience.InterfaceStability; /** * The administrative API for HBase. Obtain an instance from {@link Connection#getAdmin()} and @@ -1641,12 +1642,13 @@ default void cloneSnapshot(String snapshotName, TableName tableName) * @param snapshotName name of the snapshot to be cloned * @param tableName name of the table where the snapshot will be restored * @param restoreAcl true to clone acl into newly created table - * @param customSFT specify the StroreFileTracker used for the table + * @param customSFT specify the StoreFileTracker used for the table * @throws IOException if a remote or network exception occurs * @throws TableExistsException if table to be created already exists * @throws RestoreSnapshotException if snapshot failed to be cloned * @throws IllegalArgumentException if the specified table has not a valid name */ + @InterfaceStability.Evolving default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index e720efbdff8b..b9d02cae1758 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -224,6 +224,8 @@ private void updateTableDescriptorWithSFT() { return; } + validateSFT(); + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()){ @@ -235,6 +237,18 @@ private void updateTableDescriptorWithSFT() { tableDescriptor = builder.build(); } + private void validateSFT() { + try { + Configuration sftConfig = new Configuration(); + sftConfig.set(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); + StoreFileTrackerFactory.getTrackerClass(sftConfig); + } + catch (RuntimeException e){ + throw new UnsupportedOperationException("Specified SFT: " + customSFT + " was not recognized", + e); + } + } + @Override protected void rollbackState(final MasterProcedureEnv env, final CloneSnapshotState state) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java index b584e3dd9a6b..66cf9ea8b1e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -52,4 +53,19 @@ public void testCloneSnapshotWithCustomSFT() throws IOException, InterruptedExce TEST_UTIL.deleteTable(clonedTableName); } + + @Test + public void testCloneSnapshotWithIncorrectCustomSFT() throws IOException, InterruptedException { + TableName clonedTableName = + TableName.valueOf(getValidMethodName() + "-" + EnvironmentEdgeManager.currentTime()); + + IOException ioException = assertThrows(IOException.class, () -> { + admin.cloneSnapshot(snapshotName1, clonedTableName, false, "IncorrectSFT"); + }); + + assertEquals( + "java.lang.UnsupportedOperationException: " + + "Specified SFT: IncorrectSFT was not recognized", + ioException.getMessage()); + } } diff --git a/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb b/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb index 7c8af7d80c97..3edd16d326bf 100644 --- a/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb +++ b/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb @@ -35,7 +35,7 @@ def help hbase> clone_snapshot 'snapshotName', 'namespace:tableName', {RESTORE_ACL=>true} StoreFileTracker implementation used after restore can be set by the following command. - hbase> clone_snapshot 'snapshotName', 'namespace:tableName', {CLONE_SFT=>FILE} + hbase> clone_snapshot 'snapshotName', 'namespace:tableName', {CLONE_SFT=>'FILE'} EOF end From a73b2857544c5cc642e16bb6a46ce0086c91a5dd Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Mon, 13 Dec 2021 18:44:42 +0100 Subject: [PATCH 24/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot small improvements --- .../procedure/CloneSnapshotProcedure.java | 8 ++- .../master/snapshot/SnapshotManager.java | 34 +----------- .../StoreFileTrackerFactory.java | 37 +++++++++++++ .../master/snapshot/TestSnapshotManager.java | 52 ------------------- .../TestStoreFileTrackerFactory.java | 52 +++++++++++++++++++ 5 files changed, 97 insertions(+), 86 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index fd4766aafbbf..6bf1952b5690 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -225,8 +225,6 @@ private void updateTableDescriptorWithSFT() { return; } - validateSFT(); - TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()){ @@ -239,6 +237,10 @@ private void updateTableDescriptorWithSFT() { } private void validateSFT() { + if (StringUtils.isEmpty(customSFT)){ + return; + } + try { Configuration sftConfig = new Configuration(); sftConfig.set(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); @@ -393,6 +395,8 @@ private void prepareClone(final MasterProcedureEnv env) throws IOException { if (env.getMasterServices().getTableDescriptors().exists(tableName)) { throw new TableExistsException(tableName); } + + validateSFT(); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 2e231c3d162b..637f7c1941ef 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -869,36 +869,6 @@ public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final return procId; } - // Makes sure restoring a snapshot does not break the current SFT setup - // follows StoreUtils.createStoreConfiguration - static void checkSFTCompatibility(TableDescriptor currentTableDesc, - TableDescriptor snapshotTableDesc, Configuration baseConf) throws RestoreSnapshotException { - - for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) { - ColumnFamilyDescriptor snapCFDesc = snapshotTableDesc.getColumnFamily(cfDesc.getName()); - // if there is no counterpart in the snapshot it will be just deleted so the config does - // not matter - if (snapCFDesc != null) { - Configuration currentCompositeConf = - StoreUtils.createStoreConfiguration(baseConf, currentTableDesc, cfDesc); - Configuration snapCompositeConf = - StoreUtils.createStoreConfiguration(baseConf, snapshotTableDesc, snapCFDesc); - Class currentSFT = - StoreFileTrackerFactory.getTrackerClass(currentCompositeConf); - Class snapSFT = - StoreFileTrackerFactory.getTrackerClass(snapCompositeConf); - - //restoration is not possible if there is an SFT mismatch - if (currentSFT != snapSFT) { - throw new RestoreSnapshotException( - "Restoring Snapshot is not possible because " + " the config for column family " - + cfDesc.getNameAsString() + " has incompatible configuration. Current SFT: " - + currentSFT + " SFT from snapshot: " + snapSFT); - } - } - } - } - /** * Restore the specified snapshot. The restore will fail if the destination table has a snapshot * or restore in progress. @@ -917,8 +887,8 @@ private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableN MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); //have to check first if restoring the snapshot would break current SFT setup - checkSFTCompatibility(master.getTableDescriptors().get(tableName), snapshotTableDesc, - master.getConfiguration()); + StoreFileTrackerFactory.checkForRestoreSnapshot(master.getTableDescriptors().get(tableName), + snapshotTableDesc, master.getConfiguration()); if (master.getTableStateManager().isTableState( TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index ab9f1b333233..a57b800c86c7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreUtils; +import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -311,4 +312,40 @@ public static void checkForModifyTable(Configuration conf, TableDescriptor oldTa } } } + + /** + * Makes sure restoring a snapshot does not break the current SFT setup + * follows StoreUtils.createStoreConfiguration + * @param currentTableDesc Existing Table's TableDescriptor + * @param snapshotTableDesc Snapshot's TableDescriptor + * @param baseConf Current global configuration + * @throws RestoreSnapshotException if restore would break the current SFT setup + */ + public static void checkForRestoreSnapshot(TableDescriptor currentTableDesc, + TableDescriptor snapshotTableDesc, Configuration baseConf) throws RestoreSnapshotException { + + for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) { + ColumnFamilyDescriptor snapCFDesc = snapshotTableDesc.getColumnFamily(cfDesc.getName()); + // if there is no counterpart in the snapshot it will be just deleted so the config does + // not matter + if (snapCFDesc != null) { + Configuration currentCompositeConf = + StoreUtils.createStoreConfiguration(baseConf, currentTableDesc, cfDesc); + Configuration snapCompositeConf = + StoreUtils.createStoreConfiguration(baseConf, snapshotTableDesc, snapCFDesc); + Class currentSFT = + StoreFileTrackerFactory.getTrackerClass(currentCompositeConf); + Class snapSFT = + StoreFileTrackerFactory.getTrackerClass(snapCompositeConf); + + //restoration is not possible if there is an SFT mismatch + if (currentSFT != snapSFT) { + throw new RestoreSnapshotException( + "Restoring Snapshot is not possible because " + " the config for column family " + + cfDesc.getNameAsString() + " has incompatible configuration. Current SFT: " + + currentSFT + " SFT from snapshot: " + snapSFT); + } + } + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java index b1486f378ae8..1d96536d39c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.master.snapshot; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -29,22 +28,15 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner; import org.apache.hadoop.hbase.procedure.ProcedureCoordinator; -import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; -import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.zookeeper.KeeperException; import org.junit.ClassRule; import org.junit.Rule; @@ -205,48 +197,4 @@ private boolean isSnapshotSupported(final SnapshotManager manager) { return false; } } - - @Test public void testCheckSFTCompatibility() throws Exception { - //checking default value change on different configuration levels - Configuration conf = new Configuration(); - conf.set(StoreFileTrackerFactory.TRACKER_IMPL, "DEFAULT"); - - //creating a TD with only TableDescriptor level config - TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf("TableX")); - builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); - ColumnFamilyDescriptor cf = ColumnFamilyDescriptorBuilder.of("cf"); - builder.setColumnFamily(cf); - TableDescriptor td = builder.build(); - - //creating a TD with matching ColumnFamilyDescriptor level setting - TableDescriptorBuilder snapBuilder = - TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY")); - snapBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); - ColumnFamilyDescriptorBuilder snapCFBuilder = - ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")); - snapCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); - snapBuilder.setColumnFamily(snapCFBuilder.build()); - TableDescriptor snapTd = snapBuilder.build(); - - // adding a cf config that matches the td config is fine even when it does not match the default - SnapshotManager.checkSFTCompatibility(td, snapTd, conf); - // removing cf level config is fine when it matches the td config - SnapshotManager.checkSFTCompatibility(snapTd, td, conf); - - TableDescriptorBuilder defaultBuilder = - TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY")); - defaultBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); - ColumnFamilyDescriptorBuilder defaultCFBuilder = - ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")); - defaultCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "DEFAULT"); - defaultBuilder.setColumnFamily(defaultCFBuilder.build()); - TableDescriptor defaultTd = defaultBuilder.build(); - - assertThrows(RestoreSnapshotException.class, () -> { - SnapshotManager.checkSFTCompatibility(td, defaultTd, conf); - }); - assertThrows(RestoreSnapshotException.class, () -> { - SnapshotManager.checkSFTCompatibility(snapTd, defaultTd, conf); - }); - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java index 41f2afdfa421..af03f92a8466 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java @@ -22,9 +22,16 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,4 +62,49 @@ public void testCreateForMigration() { assertThrows(IllegalArgumentException.class, () -> StoreFileTrackerFactory .createForMigration(conf, configName, false, StoreContext.getBuilder().build())); } + + @Test + public void testCheckSFTCompatibility() throws Exception { + //checking default value change on different configuration levels + Configuration conf = new Configuration(); + conf.set(StoreFileTrackerFactory.TRACKER_IMPL, "DEFAULT"); + + //creating a TD with only TableDescriptor level config + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf("TableX")); + builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); + ColumnFamilyDescriptor cf = ColumnFamilyDescriptorBuilder.of("cf"); + builder.setColumnFamily(cf); + TableDescriptor td = builder.build(); + + //creating a TD with matching ColumnFamilyDescriptor level setting + TableDescriptorBuilder snapBuilder = + TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY")); + snapBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); + ColumnFamilyDescriptorBuilder snapCFBuilder = + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")); + snapCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); + snapBuilder.setColumnFamily(snapCFBuilder.build()); + TableDescriptor snapTd = snapBuilder.build(); + + // adding a cf config that matches the td config is fine even when it does not match the default + StoreFileTrackerFactory.checkForRestoreSnapshot(td, snapTd, conf); + // removing cf level config is fine when it matches the td config + StoreFileTrackerFactory.checkForRestoreSnapshot(snapTd, td, conf); + + TableDescriptorBuilder defaultBuilder = + TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY")); + defaultBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE"); + ColumnFamilyDescriptorBuilder defaultCFBuilder = + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf")); + defaultCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "DEFAULT"); + defaultBuilder.setColumnFamily(defaultCFBuilder.build()); + TableDescriptor defaultTd = defaultBuilder.build(); + + assertThrows(RestoreSnapshotException.class, () -> { + StoreFileTrackerFactory.checkForRestoreSnapshot(td, defaultTd, conf); + }); + assertThrows(RestoreSnapshotException.class, () -> { + StoreFileTrackerFactory.checkForRestoreSnapshot(snapTd, defaultTd, conf); + }); + } } From 5e6f4d4e3b514b78297d0ac128a30bab79641e92 Mon Sep 17 00:00:00 2001 From: BukrosSzabolcs Date: Wed, 15 Dec 2021 10:51:45 +0100 Subject: [PATCH 25/26] Update hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java Co-authored-by: Josh Elser --- .../hadoop/hbase/master/procedure/CloneSnapshotProcedure.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index 6bf1952b5690..c9b04edb929a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -245,8 +245,7 @@ private void validateSFT() { Configuration sftConfig = new Configuration(); sftConfig.set(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); StoreFileTrackerFactory.getTrackerClass(sftConfig); - } - catch (RuntimeException e){ + } catch (RuntimeException e) { throw new UnsupportedOperationException("Specified SFT: " + customSFT + " was not recognized", e); } From 7218812a2b3decafbee963c4e5ec9b39e64df5f7 Mon Sep 17 00:00:00 2001 From: Szabolcs Bukros Date: Wed, 15 Dec 2021 14:07:31 +0100 Subject: [PATCH 26/26] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot remove IS.Evolving change customSFt validation error handling rename snapshot restore check --- .../java/org/apache/hadoop/hbase/client/Admin.java | 1 - .../master/procedure/CloneSnapshotProcedure.java | 12 ++++-------- .../hbase/master/snapshot/SnapshotManager.java | 5 +---- .../storefiletracker/StoreFileTrackerFactory.java | 2 +- .../client/TestCloneSnapshotFromClientCustomSFT.java | 4 ++-- .../TestStoreFileTrackerFactory.java | 8 ++++---- 6 files changed, 12 insertions(+), 20 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index afe540c03422..6c36660dd24c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -1648,7 +1648,6 @@ default void cloneSnapshot(String snapshotName, TableName tableName) * @throws RestoreSnapshotException if snapshot failed to be cloned * @throws IllegalArgumentException if the specified table has not a valid name */ - @InterfaceStability.Evolving default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index c9b04edb929a..f6185d156037 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -241,14 +241,10 @@ private void validateSFT() { return; } - try { - Configuration sftConfig = new Configuration(); - sftConfig.set(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); - StoreFileTrackerFactory.getTrackerClass(sftConfig); - } catch (RuntimeException e) { - throw new UnsupportedOperationException("Specified SFT: " + customSFT + " was not recognized", - e); - } + //if customSFT is invalid getTrackerClass will throw a RuntimeException + Configuration sftConfig = new Configuration(); + sftConfig.set(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); + StoreFileTrackerFactory.getTrackerClass(sftConfig); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 637f7c1941ef..76cd1bf793d9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableState; @@ -67,8 +66,6 @@ import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; -import org.apache.hadoop.hbase.regionserver.StoreUtils; -import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.User; @@ -887,7 +884,7 @@ private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableN MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); //have to check first if restoring the snapshot would break current SFT setup - StoreFileTrackerFactory.checkForRestoreSnapshot(master.getTableDescriptors().get(tableName), + StoreFileTrackerFactory.validatePreRestoreSnapshot(master.getTableDescriptors().get(tableName), snapshotTableDesc, master.getConfiguration()); if (master.getTableStateManager().isTableState( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index a57b800c86c7..61a71c20f8b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java @@ -321,7 +321,7 @@ public static void checkForModifyTable(Configuration conf, TableDescriptor oldTa * @param baseConf Current global configuration * @throws RestoreSnapshotException if restore would break the current SFT setup */ - public static void checkForRestoreSnapshot(TableDescriptor currentTableDesc, + public static void validatePreRestoreSnapshot(TableDescriptor currentTableDesc, TableDescriptor snapshotTableDesc, Configuration baseConf) throws RestoreSnapshotException { for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java index 66cf9ea8b1e5..53b7f58d9bb6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java @@ -64,8 +64,8 @@ public void testCloneSnapshotWithIncorrectCustomSFT() throws IOException, Interr }); assertEquals( - "java.lang.UnsupportedOperationException: " + - "Specified SFT: IncorrectSFT was not recognized", + "java.lang.RuntimeException: java.lang.RuntimeException: " + + "java.lang.ClassNotFoundException: Class IncorrectSFT not found", ioException.getMessage()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java index af03f92a8466..91038e9fe176 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java @@ -87,9 +87,9 @@ public void testCheckSFTCompatibility() throws Exception { TableDescriptor snapTd = snapBuilder.build(); // adding a cf config that matches the td config is fine even when it does not match the default - StoreFileTrackerFactory.checkForRestoreSnapshot(td, snapTd, conf); + StoreFileTrackerFactory.validatePreRestoreSnapshot(td, snapTd, conf); // removing cf level config is fine when it matches the td config - StoreFileTrackerFactory.checkForRestoreSnapshot(snapTd, td, conf); + StoreFileTrackerFactory.validatePreRestoreSnapshot(snapTd, td, conf); TableDescriptorBuilder defaultBuilder = TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY")); @@ -101,10 +101,10 @@ public void testCheckSFTCompatibility() throws Exception { TableDescriptor defaultTd = defaultBuilder.build(); assertThrows(RestoreSnapshotException.class, () -> { - StoreFileTrackerFactory.checkForRestoreSnapshot(td, defaultTd, conf); + StoreFileTrackerFactory.validatePreRestoreSnapshot(td, defaultTd, conf); }); assertThrows(RestoreSnapshotException.class, () -> { - StoreFileTrackerFactory.checkForRestoreSnapshot(snapTd, defaultTd, conf); + StoreFileTrackerFactory.validatePreRestoreSnapshot(snapTd, defaultTd, conf); }); } }