From 1d4b08419daab531484c9b63bd1c4ccf15599a4d Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 29 Jul 2021 18:35:19 +0800 Subject: [PATCH 01/32] HBASE-26064 Introduce a StoreFileTracker to abstract the store file tracking logic Signed-off-by: Wellington Chevreuil Conflicts: hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java --- .../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 | 732 ++++-------------- .../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 | 9 +- .../hbase/regionserver/TestHRegion.java | 4 +- .../hadoop/hbase/regionserver/TestHStore.java | 33 +- .../TestRegionMergeTransactionOnCluster.java | 6 +- .../TestStoreFileRefresherChore.java | 3 +- .../regionserver/TestStoreScannerClosure.java | 8 +- .../regionserver/TestStripeStoreEngine.java | 5 +- .../compactions/TestDateTieredCompactor.java | 16 +- .../TestStripeCompactionPolicy.java | 12 +- .../compactions/TestStripeCompactor.java | 16 +- 36 files changed, 1272 insertions(+), 736 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 75e28112fe5c..70f883ff5c26 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 @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.Date; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -80,17 +79,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 cc610e5a7377..a52ce2b05c0c 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 @@ -23,7 +23,6 @@ import java.util.ArrayList; import java.util.Date; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -115,8 +114,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 3b50109901bc..ce2b38bd8966 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 @@ -27,7 +27,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 021382734ea3..b27207432ac7 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; @@ -163,18 +156,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, // rows that has cells from both memstore and files (or only files) private LongAdder mixedRowReadsCount = new LongAdder(); - 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 +264,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); @@ -313,7 +288,6 @@ protected HStore(final HRegion region, final ColumnFamilyDescriptor family, this, memstore.getClass().getSimpleName(), policyName, verifyBulkLoads, parallelPutCountPrintThreshold, family.getDataBlockEncoding(), family.getCompressionType()); - cacheOnWriteLogged = false; } private StoreContext initializeStoreContext(ColumnFamilyDescriptor family) throws IOException { @@ -510,105 +484,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.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 +503,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 +527,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 +535,7 @@ public void add(final Cell cell, MemStoreSizing memstoreSizing) { } this.memstore.add(cell, memstoreSizing); } finally { - lock.readLock().unlock(); + storeEngine.readUnlock(); currentParallelPutCount.decrementAndGet(); } } @@ -738,7 +544,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 +552,7 @@ public void add(final Iterable cells, MemStoreSizing memstoreSizing) { } memstore.add(cells, memstoreSizing); } finally { - lock.readLock().unlock(); + storeEngine.readUnlock(); currentParallelPutCount.decrementAndGet(); } } @@ -869,17 +675,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 +692,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 +768,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.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 +809,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 +855,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 +897,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 +943,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 +1026,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 +1146,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 +1216,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 +1356,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 +1364,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 +1373,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 +1420,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 +1493,7 @@ public Optional requestCompaction(int priority, request.setTracker(tracker); } } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } if (LOG.isDebugEnabled()) { @@ -1892,7 +1526,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 +1538,7 @@ private void removeUnneededFiles() throws IOException { } } } finally { - this.lock.readLock().unlock(); + this.storeEngine.readUnlock(); } if (CollectionUtils.isEmpty(delSfs)) { @@ -1912,8 +1546,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 +1568,6 @@ private 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 +1613,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 +1626,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 +1659,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 +1669,7 @@ public KeyValueScanner getScanner(Scan scan, final NavigableSet targetCo } return createScanner(scan, scanInfo, targetCols, readPt); } finally { - lock.readLock().unlock(); + storeEngine.readUnlock(); } } @@ -2085,7 +1699,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 +1724,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 +1790,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 +1874,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 +1931,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 +1954,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 +1982,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 @@ -2405,7 +1998,7 @@ public void replayFlush(List fileNames, boolean dropMemstoreSnapshot) if (dropMemstoreSnapshot && snapshot != null) { snapshotId = snapshot.getId(); } - HStore.this.updateStorefiles(storeFiles, snapshotId); + HStore.this.completeFlush(storeFiles, snapshotId); } /** @@ -2414,7 +2007,7 @@ public void replayFlush(List fileNames, boolean dropMemstoreSnapshot) @Override public void abort() throws IOException { if (snapshot != null) { - HStore.this.updateStorefiles(Collections.emptyList(), snapshot.getId()); + HStore.this.completeFlush(Collections.emptyList(), snapshot.getId()); } } } @@ -2577,7 +2170,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 = @@ -2589,7 +2182,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); @@ -2724,12 +2317,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); } @Override 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 67eb375ee6f6..1095854273a0 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 @@ -69,10 +69,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 7f70e0230ff5..8178fb1df164 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; @@ -38,6 +37,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; @@ -60,6 +60,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; /** @@ -260,29 +261,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 f765b350790c..2112b97c741f 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 @@ -101,7 +101,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 8adffd329c78..c869e5e5a88c 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 @@ -210,11 +210,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 7d6d624c1502..f8259051c471 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 d36eb11fb2ec..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 @@ -21,7 +21,7 @@ import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -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(HRegionInfo.FIRST_META_REGIONINFO); - StoreEngine se = StoreEngine.create(mockStore, conf, CellComparatorImpl.COMPARATOR); + Mockito.when(mockStore.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO); + 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 d494dd34762a..131f3452a0c0 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 @@ -5746,7 +5746,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); @@ -7715,7 +7715,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 e171a4fa6d27..b4d3b8a02e6a 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; @@ -315,7 +315,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 { @@ -327,9 +327,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))); @@ -1027,19 +1029,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) { @@ -1721,7 +1723,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)); @@ -1800,8 +1802,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()); @@ -3348,7 +3352,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 89d64f789772..a5555ae09e1d 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 @@ -244,7 +244,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 @@ -253,7 +253,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; @@ -272,7 +272,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 30a005d0557c..daec3a2ec57b 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 73d20ec591d2..515e1dc4be51 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 @@ -30,7 +30,6 @@ import java.util.TreeSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -62,6 +61,7 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + /** * This test tests whether parallel {@link StoreScanner#close()} and * {@link StoreScanner#updateReaders(List, List)} works perfectly ensuring @@ -85,7 +85,6 @@ public class TestStoreScannerClosure { private static CacheConfig cacheConf; private static FileSystem fs; private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static String ROOT_DIR = TEST_UTIL.getDataTestDir("TestHFile").toString(); private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE, Long.MAX_VALUE, KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.getInstance(), false); private final static byte[] fam = Bytes.toBytes("cf_1"); @@ -127,13 +126,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 7a93f8972fbb..d6cc5692bf19 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 @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy; @@ -118,8 +119,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(HRegionInfo.FIRST_META_REGIONINFO); + 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 92ba76d4117c..dd844080aa82 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; @@ -44,11 +41,14 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +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; @@ -93,6 +93,7 @@ public static Iterable data() { @Parameter public boolean usePrivateReaders; + @SuppressWarnings({ "rawtypes", "unchecked" }) private DateTieredCompactor createCompactor(StoreFileWritersCapture writers, final KeyValue[] input, List storefiles) throws Exception { Configuration conf = HBaseConfiguration.create(); @@ -107,11 +108,10 @@ private DateTieredCompactor createCompactor(StoreFileWritersCapture writers, when(store.getScanInfo()).thenReturn(si); when(store.areWritesEnabled()).thenReturn(true); when(store.getFileSystem()).thenReturn(mock(FileSystem.class)); - when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME)); - when(store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers); - when(store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyString())).thenAnswer(writers); + when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.newBuilder(TABLE_NAME).build()); + 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 318f326c94c0..0542859c0857 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 @@ -31,7 +31,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; @@ -61,6 +60,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; @@ -68,6 +68,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; @@ -903,12 +904,9 @@ private StripeCompactor createCompactor() throws Exception { when(info.getRegionNameAsString()).thenReturn("testRegion"); when(store.getColumnFamilyDescriptor()).thenReturn(col); 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 6e8b19fdd013..0ae584c5aa9b 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; @@ -42,10 +39,13 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +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; @@ -192,6 +192,7 @@ private void verifySizeCompaction(KeyValue[] input, int targetCount, long target writers.verifyBoundaries(boundaries.toArray(new byte[][] {})); } + @SuppressWarnings({ "rawtypes", "unchecked" }) private StripeCompactor createCompactor(StoreFileWritersCapture writers, KeyValue[] input) throws Exception { Configuration conf = HBaseConfiguration.create(); @@ -206,11 +207,10 @@ private StripeCompactor createCompactor(StoreFileWritersCapture writers, KeyValu when(store.getScanInfo()).thenReturn(si); when(store.areWritesEnabled()).thenReturn(true); when(store.getFileSystem()).thenReturn(mock(FileSystem.class)); - when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME)); - when(store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers); - when(store.createWriterInTmp(anyLong(), any(), anyBoolean(), - anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyString())).thenAnswer(writers); + when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.newBuilder(TABLE_NAME).build()); + 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 177cbf919ceb51d09d21e0b87272145cd8786e8f Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 26 Aug 2021 18:51:12 +0800 Subject: [PATCH 02/32] HBASE-25988 Store the store file list by a file (#3578) Signed-off-by: Wellington Chevreuil --- .../src/main/protobuf/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/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/StoreFileTracker.proto b/hbase-protocol-shaded/src/main/protobuf/StoreFileTracker.proto new file mode 100644 index 000000000000..2a269ea4ac4e --- /dev/null +++ b/hbase-protocol-shaded/src/main/protobuf/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..9129dc3f1281 --- /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.HBaseTestingUtility; +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 HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + 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 = + HBaseTestingUtility.createRegionAndWAL(RI, UTIL.getDataTestDir(name.getMethodName()), conf, TD); + } + + @After + public void tearDown() throws IOException { + if (region != null) { + HBaseTestingUtility.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 37e0b0e898c6e1df1c51ff53959ddb90f30a411c Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Thu, 17 Mar 2022 17:35:36 -0700 Subject: [PATCH 03/32] HBASE-26079 Use StoreFileTracker when splitting and merging (#3617) Signed-off-by: Duo Zhang Conflicts: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java --- .../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 | 34 ++- .../hbase/regionserver/TestHStoreFile.java | 23 +- .../TestMergesSplitsAddToTracker.java | 267 ++++++++++++++++++ .../regionserver/TestStripeStoreEngine.java | 1 + .../TestStoreFileTracker.java | 56 ++++ 12 files changed, 476 insertions(+), 50 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 347c7d2e7d81..75844991f8d7 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; @@ -57,6 +56,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; @@ -588,30 +589,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); @@ -623,11 +629,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 888ab531c4e8..ba7aa971bd19 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,14 +211,17 @@ 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 { List> procedures = TEST_UTIL.getHBaseCluster().getMaster().getProcedures(); if(procedures.size()>0) { - Procedure splitProc = procedures.stream(). + Procedure splitProc = procedures.stream(). filter(p -> p instanceof SplitTableRegionProcedure).findFirst().get(); int count = 0; while ((splitProc.isWaiting() || splitProc.isRunnable()) && count < attempts) { @@ -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 2bbbc5cb7c47..9002bdcb07f7 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 @@ -51,12 +51,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; @@ -71,6 +73,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; @@ -88,7 +92,6 @@ import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.base.Joiner; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -134,11 +137,7 @@ public static void tearDownAfterClass() { /** * Write a file and then assert that we can read from top and bottom halves using two -<<<<<<< HEAD - * HalfMapFiles, as well as one HalfMapFile and one HFileLink file. -======= * HalfMapFiles. ->>>>>>> 16116fa35e... HBASE-24510 Remove HBaseTestCase and GenericTestUtils (#1859) */ @Test public void testBasicHalfAndHFileLinkMapFile() throws Exception { @@ -1062,7 +1061,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..6a9e08f2a0c9 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java @@ -0,0 +1,267 @@ +/** + * 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.HBaseTestingUtility; +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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + 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 + 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 + 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); + } + + private void split(TableName table, byte[] splitKey) throws IOException { + TEST_UTIL.getAdmin().split(table, splitKey); + // wait until split is done + TEST_UTIL.waitFor(30000, () -> TEST_UTIL.getHBaseCluster().getRegions(table).size() == 2); + } + + @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 d6cc5692bf19..e30c9d9f5015 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 @@ -121,6 +121,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 727d6135063aba3ed0391afce8c3f766fe0818ef Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 9 Sep 2021 12:11:25 +0800 Subject: [PATCH 04/32] HBASE-26224 Introduce a MigrationStoreFileTracker to support migrating from different store file tracker implementations (#3656) Signed-off-by: Wellington Chevreuil Conflicts: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java --- .../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..567adf040f8e --- /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.HBaseTestingUtility; +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 HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + 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 = HBaseTestingUtility.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 1242f95eddb84c2f042ddc63458a298277f75196 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Sun, 12 Sep 2021 14:14:03 +0100 Subject: [PATCH 05/32] HBASE-26246 Persist the StoreFileTracker configurations to TableDescriptor when creating table (#3666) Signed-off-by: Duo Zhang --- .../hbase/client/TableDescriptorBuilder.java | 4 +++ .../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 | 7 ++++++ .../hbase/client/TestAsyncTableAdminApi3.java | 8 +++++- .../MasterProcedureTestingUtility.java | 7 ++++++ .../procedure/TestCreateTableProcedure.java | 17 +++++++++++++ .../TestStoreFileTracker.java | 13 +++++++--- 14 files changed, 117 insertions(+), 14 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index c2ca5a1d59b4..823b61e5ceb8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -581,6 +581,10 @@ public TableDescriptorBuilder setValue(final byte[] key, final byte[] value) { return this; } + public String getValue(String key) { + return desc.getValue(key); + } + /** * Sets replication scope all & only the columns already in the builder. Columns added later won't * be backfilled with replication scope. 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 4242ed662d0c..219821f2b43a 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; @@ -31,10 +32,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.util.CommonFSUtils; import org.apache.hadoop.hbase.util.FSTableDescriptors; @@ -265,6 +268,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 d43dbe060d01..2235cb326089 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 @@ -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.assertNotEquals; @@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.LoadBalancer; +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; @@ -420,6 +422,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 42d0118ba657..67b19c0853d9 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,10 +18,12 @@ 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; import static org.junit.Assert.fail; + import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -38,6 +40,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.LoadBalancer; +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; @@ -409,6 +412,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 f8b37a93d48c..b30a08d28c43 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; @@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor; +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; @@ -149,7 +151,11 @@ public void testGetTableDescriptor() throws Exception { admin.createTable(desc).join(); ModifyableTableDescriptor modifyableDesc = ((ModifyableTableDescriptor) desc); TableDescriptor confirmedHtd = admin.getDescriptor(tableName).get(); - assertEquals(0, modifyableDesc.compareTo((ModifyableTableDescriptor) confirmedHtd)); + //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)); } @Test 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 94254d97c558..1748f102896a 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; @@ -58,6 +59,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; @@ -224,6 +226,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..80d0ec6ebd9e 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 @@ -23,7 +23,6 @@ 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; @@ -38,15 +37,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 7eaff3ad177195014f5e0bf24a73d51279f2ef3a Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Tue, 14 Sep 2021 16:28:21 +0800 Subject: [PATCH 06/32] HBASE-26248 Should find a suitable way to let users specify the store file tracker implementation (#3665) Signed-off-by: Wellington Chevreuil Conflicts: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java --- .../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 | 7 +- .../TestStoreFileTrackerFactory.java | 58 +++++++++++ 14 files changed, 204 insertions(+), 57 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 2235cb326089..a8e4b7e59504 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 @@ -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.assertNotEquals; @@ -423,8 +423,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 67b19c0853d9..8792f894436f 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; @@ -413,8 +413,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 b30a08d28c43..d2550fc4a6e5 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; @@ -152,8 +152,8 @@ public void testGetTableDescriptor() throws Exception { ModifyableTableDescriptor modifyableDesc = ((ModifyableTableDescriptor) desc); 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 1748f102896a..210b2755af90 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; @@ -229,8 +229,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 6a9e08f2a0c9..435fa26f7551 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; @@ -73,7 +73,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 567adf040f8e..c6f51ff61342 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 = HBaseTestingUtility.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 9129dc3f1281..ee86d7054360 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,10 +71,9 @@ public class TestRegionWithFileBasedStoreFileTracker { @Before public void setUp() throws IOException { Configuration conf = new Configuration(UTIL.getConfiguration()); - conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, FileBasedStoreFileTracker.class, - StoreFileTracker.class); - region = - HBaseTestingUtility.createRegionAndWAL(RI, UTIL.getDataTestDir(name.getMethodName()), conf, TD); + conf.set(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()); + region = HBaseTestingUtility.createRegionAndWAL(RI, UTIL.getDataTestDir(name.getMethodName()), + conf, TD); } @After 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 9d5dd034850deb11074f457c42568730561034ce Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 15 Sep 2021 23:00:03 +0800 Subject: [PATCH 07/32] 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 75844991f8d7..6e5151c671cd 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 @@ -615,8 +615,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 219821f2b43a..80ed96a77f24 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 @@ -257,15 +257,17 @@ private boolean prepareCreate(final MasterProcedureEnv env) throws IOException { return false; } + // 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 9892dba6b181..e58fc92574c9 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 @@ -37,6 +37,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.util.Bytes; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; @@ -317,6 +318,10 @@ private void prepareModify(final MasterProcedureEnv env) throws IOException { this.deleteColumnFamilyInModify = isDeleteColumnFamily(unmodifiedTableDescriptor, modifiedTableDescriptor); + + // 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..110f896df8b8 --- /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.HBaseTestingUtility; +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 HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + @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 1e127f68b83aa1df2bed433dd0467336a13f2f52 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 17 Sep 2021 09:40:44 +0800 Subject: [PATCH 08/32] 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 | 34 ++++++--- .../hbase/regionserver/TestHStoreFile.java | 6 +- .../snapshot/MobSnapshotTestingUtils.java | 74 ++++++++++--------- .../hbase/snapshot/SnapshotTestingUtils.java | 16 ++-- 11 files changed, 109 insertions(+), 111 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 6e5151c671cd..ab4020710039 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 @@ -613,9 +613,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 79e587ac44b6..368e72b3d02d 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 699c086b58bf..3c24911ce7e2 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 @@ -30,6 +30,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; @@ -92,7 +93,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 65704ce3f190..76356fe5af78 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; @@ -33,9 +34,11 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; 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; @@ -52,7 +55,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; @@ -65,7 +71,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 @@ -83,7 +90,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 @@ -110,7 +126,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,9 +134,10 @@ public void setup() throws Exception { } protected void createTable() throws Exception { - HTableDescriptor htd = new HTableDescriptor(TABLE_NAME); - htd.setRegionReplication(getNumReplicas()); - UTIL.createTable(htd, new byte[][]{TEST_FAM}, null); + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(getNumReplicas()) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, trackerImpl.name()).build(); + UTIL.createTable(htd, new byte[][] { TEST_FAM }, null); } protected int getNumReplicas() { @@ -326,7 +342,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(); @@ -411,7 +427,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 9002bdcb07f7..2b8565a22b9d 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 @@ -59,6 +59,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; @@ -1069,10 +1070,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 9ee433631210..4fb1ae2e1f04 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 HBaseTestingUtility util, - final TableName tableName, int regionReplication, - final byte[]... families) throws IOException, InterruptedException { - createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(), - regionReplication, families); + public static void createMobTable(final HBaseTestingUtility 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 HBaseTestingUtility 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 HBaseTestingUtility 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 HBaseTestingUtility 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 HBaseTestingUtility 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 HBaseTestingUtility 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(HBaseTestingUtility util, TableName tableName, - byte[][] splitKeys, int regionReplication, String cpClassName, byte[]... families) - throws IOException, InterruptedException { + public static void createMobTable(HBaseTestingUtility 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(HBaseTestingUtility 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(HBaseTestingUtility 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 HBaseTestingUtility util, - final TableName tableName, final byte[]... families) throws IOException { + public static Table createMobTable(final HBaseTestingUtility 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 HBaseTestingUtility 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 HBaseTestingUtility util, - final TableName tableName, long expectedRows) throws IOException { + public static void verifyMobRowCount(final HBaseTestingUtility 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 509842bdc60c..a19bf4f6a127 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 @@ -509,9 +509,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() { @@ -531,15 +530,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 { @@ -550,6 +547,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 9c51b691bb9f21d95927ad3736f859a04b10ea41 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Wed, 13 Oct 2021 15:48:13 +0100 Subject: [PATCH 09/32] =?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 80ed96a77f24..55e32126bdfb 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 @@ -32,7 +32,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; @@ -270,9 +269,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 80d0ec6ebd9e..4a471baa8689 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 @@ -37,7 +37,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 9287a37089ea3b9a540f0fb4ac3305ed14596884 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 21 Oct 2021 10:27:45 +0800 Subject: [PATCH 10/32] 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 73c063313dee854e7fde54294872cdc9504525a5 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Fri, 22 Oct 2021 16:56:15 +0100 Subject: [PATCH 11/32] 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 8157af99ba4b..7157fbf04d40 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 @@ -41,7 +41,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; @@ -459,56 +458,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 d6146c23e0b9a9ec9d69a6cbf4d3db359f6a4c34 Mon Sep 17 00:00:00 2001 From: GeorryHuang Date: Sat, 6 Nov 2021 22:20:12 +0800 Subject: [PATCH 12/32] HBASE-26263 [Rolling Upgrading] Persist the StoreFileTracker configurations to TableDescriptor for existing tables (#3700) Signed-off-by: Duo Zhang Reviewed-by: Wellington Ramos Chevreuil --- .../hbase/client/TableDescriptorBuilder.java | 16 ++ .../src/main/protobuf/MasterProcedure.proto | 11 ++ .../apache/hadoop/hbase/master/HMaster.java | 6 + .../master/migrate/RollingUpgradeChore.java | 130 ++++++++++++++ .../ModifyTableDescriptorProcedure.java | 161 ++++++++++++++++++ .../MigrateStoreFileTrackerProcedure.java | 48 ++++++ .../migrate/TestMigrateStoreFileTracker.java | 108 ++++++++++++ 7 files changed, 480 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/master/procedure/ModifyTableDescriptorProcedure.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-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index 823b61e5ceb8..d9a15154a313 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -440,6 +440,11 @@ public TableDescriptorBuilder modifyColumnFamily(final ColumnFamilyDescriptor fa return this; } + public TableDescriptorBuilder removeValue(final String key) { + desc.removeValue(key); + return this; + } + public TableDescriptorBuilder removeValue(Bytes key) { desc.removeValue(key); return this; @@ -788,6 +793,17 @@ private static Bytes toBytesOrNull(T t, Function f) { } } + /** + * Remove metadata represented by the key from the {@link #values} map + * + * @param key Key whose key and value we're to remove from TableDescriptor + * parameters. + * @return the modifyable TD + */ + public ModifyableTableDescriptor removeValue(final String key) { + return setValue(key, (String) null); + } + /** * Remove metadata represented by the key from the {@link #values} map * diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index fc97290add48..66d1640763fc 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -584,3 +584,14 @@ enum ClaimReplicationQueuesState { CLAIM_REPLICATION_QUEUES_DISPATCH = 1; CLAIM_REPLICATION_QUEUES_FINISH = 2; } + + +enum ModifyTableDescriptorState { + MODIFY_TABLE_DESCRIPTOR_PREPARE = 1; + MODIFY_TABLE_DESCRIPTOR_UPDATE = 2; +} + +message ModifyTableDescriptorStateData { + required TableSchema unmodified_table_schema = 1; + optional TableSchema modified_table_schema = 2; +} 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 91be0c047251..a13cd0cfcbb4 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 @@ -133,6 +133,7 @@ import org.apache.hadoop.hbase.master.http.api_v1.ResourceConfigFactory; 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; @@ -360,6 +361,7 @@ public class HMaster extends HRegionServer implements MasterServices { private ExpiredMobFileCleanerChore expiredMobFileCleanerChore; private MobCompactionChore mobCompactChore; private MasterMobCompactionThread mobCompactThread; + private RollingUpgradeChore rollingUpgradeChore; // used to synchronize the mobCompactionStates private final IdLock mobCompactionLock = new IdLock(); // save the information of mob compactions in tables. @@ -1229,6 +1231,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) LOG.debug("Balancer post startup initialization complete, took " + ( (EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds"); } + + this.rollingUpgradeChore = new RollingUpgradeChore(this); + getChoreService().scheduleChore(rollingUpgradeChore); } private void createMissingCFsInMetaDuringUpgrade( @@ -1712,6 +1717,7 @@ private 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/master/procedure/ModifyTableDescriptorProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java new file mode 100644 index 000000000000..4f491421854a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java @@ -0,0 +1,161 @@ +/** + * 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 java.io.IOException; +import java.util.Optional; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableDescriptorState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableDescriptorStateData; + +/** + * The procedure will only update the table descriptor without reopening all the regions. + *

+ * It is usually used for migrating when upgrading, where we need to add something into the table + * descriptor, such as the rs group information. + */ +@InterfaceAudience.Private +public abstract class ModifyTableDescriptorProcedure + extends AbstractStateMachineTableProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(ModifyTableDescriptorProcedure.class); + + private TableDescriptor unmodifiedTableDescriptor; + private TableDescriptor modifiedTableDescriptor; + + protected ModifyTableDescriptorProcedure() { + } + + protected ModifyTableDescriptorProcedure(MasterProcedureEnv env, TableDescriptor unmodified) { + super(env); + this.unmodifiedTableDescriptor = unmodified; + } + + @Override + public TableName getTableName() { + return unmodifiedTableDescriptor.getTableName(); + } + + @Override + public TableOperationType getTableOperationType() { + return TableOperationType.EDIT; + } + + /** + * Sub class should implement this method to modify the table descriptor, such as storing the rs + * group information. + *

+ * Since the migrating is asynchronouns, it is possible that users have already changed the rs + * group for a table, in this case we do not need to modify the table descriptor any more, then + * you could just return {@link Optional#empty()}. + */ + protected abstract Optional modify(MasterProcedureEnv env, + TableDescriptor current) throws IOException; + + @Override + protected Flow executeFromState(MasterProcedureEnv env, ModifyTableDescriptorState state) + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { + try { + switch (state) { + case MODIFY_TABLE_DESCRIPTOR_PREPARE: + Optional modified = modify(env, unmodifiedTableDescriptor); + if (modified.isPresent()) { + modifiedTableDescriptor = modified.get(); + setNextState(ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_UPDATE); + return Flow.HAS_MORE_STATE; + } else { + // do not need to modify + return Flow.NO_MORE_STATE; + } + case MODIFY_TABLE_DESCRIPTOR_UPDATE: + env.getMasterServices().getTableDescriptors().update(modifiedTableDescriptor); + return Flow.NO_MORE_STATE; + default: + throw new UnsupportedOperationException("unhandled state=" + state); + } + } catch (IOException e) { + if (isRollbackSupported(state)) { + setFailure("master-modify-table-descriptor", e); + } else { + LOG.warn("Retriable error trying to modify table descriptor={} (in state={})", + getTableName(), state, e); + } + } + return Flow.HAS_MORE_STATE; + } + + @Override + protected void rollbackState(MasterProcedureEnv env, ModifyTableDescriptorState state) + throws IOException, InterruptedException { + if (state == ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_PREPARE) { + return; + } + throw new UnsupportedOperationException("unhandled state=" + state); + } + + @Override + protected boolean isRollbackSupported(ModifyTableDescriptorState state) { + return state == ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_PREPARE; + } + + @Override + protected ModifyTableDescriptorState getState(int stateId) { + return ModifyTableDescriptorState.forNumber(stateId); + } + + @Override + protected int getStateId(ModifyTableDescriptorState state) { + return state.getNumber(); + } + + @Override + protected ModifyTableDescriptorState getInitialState() { + return ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_PREPARE; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + ModifyTableDescriptorStateData.Builder builder = ModifyTableDescriptorStateData.newBuilder() + .setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor)); + if (modifiedTableDescriptor != null) { + builder.setModifiedTableSchema(ProtobufUtil.toTableSchema(modifiedTableDescriptor)); + } + serializer.serialize(builder.build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + ModifyTableDescriptorStateData data = + serializer.deserialize(ModifyTableDescriptorStateData.class); + unmodifiedTableDescriptor = ProtobufUtil.toTableDescriptor(data.getUnmodifiedTableSchema()); + if (data.hasModifiedTableSchema()) { + modifiedTableDescriptor = ProtobufUtil.toTableDescriptor(data.getModifiedTableSchema()); + } + } +} \ No newline at end of file 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..4f00184a107f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java @@ -0,0 +1,108 @@ +/* + * + * 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.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.TableName; +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 HBaseTestingUtility HTU; + private Configuration conf; + private HTableDescriptor 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 HBaseTestingUtility(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.createTableDescriptor(tables[i]); + tableDescriptor.addFamily(new HColumnDescriptor(fam)); + 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 43997fa7ee7cb778cb1f132a5fe551090e8589f2 Mon Sep 17 00:00:00 2001 From: BukrosSzabolcs Date: Tue, 9 Nov 2021 17:19:00 +0100 Subject: [PATCH 13/32] 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 Conflicts: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java --- .../hbase/mob/DefaultMobStoreCompactor.java | 17 +- .../regionserver/AbstractMultiFileWriter.java | 6 +- .../regionserver/BrokenStoreFileCleaner.java | 202 ++++++++++++++++ .../DateTieredMultiFileWriter.java | 2 +- .../hbase/regionserver/HRegionFileSystem.java | 2 +- .../hbase/regionserver/HRegionServer.java | 26 ++ .../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 +- .../TestBrokenStoreFileCleaner.java | 226 ++++++++++++++++++ .../regionserver/TestCompactorMemLeak.java | 4 +- .../TestStoreFileTracker.java | 1 - 22 files changed, 566 insertions(+), 33 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 70f883ff5c26..31da0c816d75 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 @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; -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.InternalScanner; @@ -52,6 +51,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + /** * Compact passed set of files in the mob-enabled column family. */ @@ -154,7 +155,6 @@ public List compact(CompactionRequestImpl request, ThroughputController th * the scanner to filter the deleted cells. * @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. @@ -163,10 +163,9 @@ public List compact(CompactionRequestImpl request, ThroughputController th * @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 bytesWrittenProgressForCloseCheck = 0; long bytesWrittenProgressForLog = 0; long bytesWrittenProgressForShippedCall = 0; // Since scanner.next() can return 'false' but still be delivering data, @@ -369,4 +368,14 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel progress.complete(); return true; } + + + @Override + protected List commitWriter(FileDetails fd, + CompactionRequestImpl request) throws IOException { + List newFiles = Lists.newArrayList(writer.getPath()); + writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); + writer.close(); + return newFiles; + } } 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 f2e74892ce2b..2bef9388021f 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 @@ -548,6 +548,8 @@ public class HRegionServer extends Thread implements */ protected final ConfigurationManager configurationManager; + private BrokenStoreFileCleaner brokenStoreFileCleaner; + @InterfaceAudience.Private CompactedHFilesDischarger compactedFileDischarger; @@ -2167,6 +2169,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. @@ -2247,6 +2252,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 = (ThreadLocalRandom.current().nextDouble() - 0.5D) * brokenStoreFileCleanerDelayJitter; + long jitterValue = Math.round(brokenStoreFileCleanerDelay * jitterRate); + this.brokenStoreFileCleaner = + new BrokenStoreFileCleaner((int) (brokenStoreFileCleanerDelay + jitterValue), + brokenStoreFileCleanerPeriod, this, conf, this); + registerConfigurationObservers(); } @@ -4039,4 +4060,9 @@ public Iterator getBootstrapNodes() { public MetaRegionLocationCache getMetaRegionLocationCache() { return this.metaRegionLocationCache; } + + @InterfaceAudience.Private + public BrokenStoreFileCleaner getBrokenStoreFileCleaner(){ + return 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 b27207432ac7..901615606713 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 @@ -1153,6 +1153,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 8178fb1df164..a821a90af50e 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; @@ -36,6 +39,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; @@ -91,6 +95,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; @@ -323,7 +329,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; @@ -347,8 +352,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 " @@ -368,24 +378,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 @@ -393,7 +402,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; @@ -536,4 +545,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/regionserver/TestBrokenStoreFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java new file mode 100644 index 000000000000..eb5e6c7fe3b7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java @@ -0,0 +1,226 @@ +/** + * 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.HBaseTestingUtility; +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 HBaseTestingUtility testUtil = new HBaseTestingUtility(); + 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 9a6e96a9bda4..1b76c52cd6e3 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 4a471baa8689..b4c70961841b 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 @@ -44,7 +44,6 @@ public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreC } else { LOG.info("ctx.getRegionFileSystem() returned null. Leaving storeId null."); } - } @Override From 062ee1d7bac4d30bee7f401fdc478427fb19b597 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Fri, 19 Nov 2021 12:16:29 +0000 Subject: [PATCH 14/32] =?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 | 115 ++++++------------ .../access/SnapshotScannerHDFSAclHelper.java | 4 +- .../hbase/master/TestMasterFileSystem.java | 29 +---- .../procedure/TestDeleteTableProcedure.java | 66 ---------- 5 files changed, 53 insertions(+), 191 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 55e32126bdfb..441fddbfe7ce 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 @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; @@ -316,41 +315,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 8322383686c8..46144867d54d 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,7 @@ 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; @@ -52,11 +49,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableState; -import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; @InterfaceAudience.Private public class DeleteTableProcedure @@ -278,92 +276,59 @@ 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."); + // Archive regions from FS (temp directory) + if (archive) { + List regionDirList = new ArrayList<>(); + for (RegionInfo region : regions) { + if (RegionReplicaUtil.isDefaultReplica(region)) { + regionDirList.add(FSUtils.getRegionDirFromTableDir(tableDir, region)); + List mergeRegions = MetaTableAccessor + .getMergeRegions(env.getMasterServices().getConnection(), region.getRegionName()); + if (!CollectionUtils.isEmpty(mergeRegions)) { + mergeRegions.stream().forEach( + r -> regionDirList.add(FSUtils.getRegionDirFromTableDir(tableDir, r))); + } + } + } + HFileArchiver + .archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(), tableDir, + regionDirList); + if (!regionDirList.isEmpty()) { + LOG.debug("Archived {} regions", tableName); + } } - 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); - } - fs.delete(tempTableDir, true); + // 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); } - // Move the table in /hbase/.tmp - if (!fs.rename(tableDir, tempTableDir)) { - throw new IOException("Unable to move '" + tableDir + "' to temp '" + tempTableDir + "'"); + // Delete table directory from FS + if (!fs.delete(tableDir, true) && fs.exists(tableDir)) { + throw new IOException("Couldn't delete " + tableDir); } - } - // Archive regions from FS (temp directory) - if (archive) { - List regionDirList = new ArrayList<>(); - for (RegionInfo region : regions) { - if (RegionReplicaUtil.isDefaultReplica(region)) { - regionDirList.add(FSUtils.getRegionDirFromTableDir(tempTableDir, region)); - List mergeRegions = MetaTableAccessor - .getMergeRegions(env.getMasterServices().getConnection(), region.getRegionName()); - if (!CollectionUtils.isEmpty(mergeRegions)) { - mergeRegions.stream() - .forEach(r -> regionDirList.add(FSUtils.getRegionDirFromTableDir(tempTableDir, r))); - } + // 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); } } - HFileArchiver.archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(), tempTableDir, - regionDirList); - if (!regionDirList.isEmpty()) { - LOG.debug("Archived {} regions", tableName); - } - } - - // 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 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 ffe8dab579ef..fbdc6381d7ad 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 @@ -478,8 +478,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 63d303dc710a..1461c0612b5c 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 899c700ace6304623f23325ac769336dbf7ff7a9 Mon Sep 17 00:00:00 2001 From: BukrosSzabolcs Date: Wed, 15 Dec 2021 20:09:03 -0500 Subject: [PATCH 15/32] HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot Closes #3851 Signed-off-by: Duo Zhang Signed-off-by: Josh Elser --- .../org/apache/hadoop/hbase/client/Admin.java | 42 ++++++++++- .../hadoop/hbase/client/AsyncAdmin.java | 14 +++- .../hadoop/hbase/client/AsyncHBaseAdmin.java | 6 +- .../hbase/client/ColumnFamilyDescriptor.java | 5 ++ .../client/ColumnFamilyDescriptorBuilder.java | 6 ++ .../hadoop/hbase/client/HBaseAdmin.java | 26 ++++--- .../hbase/client/RawAsyncHBaseAdmin.java | 24 ++++--- .../src/main/protobuf/Master.proto | 1 + .../src/main/protobuf/MasterProcedure.proto | 1 + .../apache/hadoop/hbase/master/HMaster.java | 25 +++---- .../hbase/master/MasterRpcServices.java | 2 +- .../procedure/CloneSnapshotProcedure.java | 54 +++++++++++++- .../procedure/RestoreSnapshotProcedure.java | 7 +- .../master/snapshot/SnapshotManager.java | 27 ++++--- .../StoreFileTrackerFactory.java | 39 +++++++++- .../hbase/snapshot/RestoreSnapshotHelper.java | 11 +-- .../TestCloneSnapshotFromClientCustomSFT.java | 72 +++++++++++++++++++ .../TestStoreFileTrackerFactory.java | 52 ++++++++++++++ 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 +- 22 files changed, 365 insertions(+), 64 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 a3a51071b338..36ebc9e3b391 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 @@ -70,6 +70,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 @@ -2410,7 +2411,25 @@ default void cloneSnapshot(byte[] snapshotName, TableName tableName) */ 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 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 + */ + default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, + String customSFT) + throws IOException, TableExistsException, RestoreSnapshotException { + get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT), getSyncWaitTimeout(), + TimeUnit.MILLISECONDS); } /** @@ -2457,8 +2476,25 @@ 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 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 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/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index 85d545505e99..42c3f049867e 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 @@ -872,8 +872,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 customSFT specify the StroreFileTracker used for the table + */ CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl); + 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 db720f3f68ed..16ebd1587b47 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 @@ -483,14 +483,14 @@ public CompletableFuture restoreSnapshot(String snapshotName) { @Override public CompletableFuture restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, - boolean 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 customSFT) { + return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl, customSFT)); } @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 d1c9264fc0a0..5dccd0b40c5c 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 @@ -673,6 +673,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/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 0704d518d076..b860bdc38cd2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -108,9 +108,11 @@ import org.apache.yetus.audience.InterfaceStability; 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.protobuf.ServiceException; import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos; @@ -2628,7 +2630,7 @@ public void restoreSnapshot(final String snapshotName, final boolean takeFailSaf try { // Restore snapshot get( - internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl), + internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl, null), syncWaitTimeout, TimeUnit.MILLISECONDS); } catch (IOException e) { @@ -2637,7 +2639,7 @@ public void restoreSnapshot(final String snapshotName, final boolean takeFailSaf if (takeFailSafeSnapshot) { try { get( - internalRestoreSnapshotAsync(failSafeSnapshotSnapshotName, tableName, restoreAcl), + internalRestoreSnapshotAsync(failSafeSnapshotSnapshotName, tableName, restoreAcl, null), syncWaitTimeout, TimeUnit.MILLISECONDS); String msg = "Restore snapshot=" + snapshotName + @@ -2680,16 +2682,17 @@ public Future restoreSnapshotAsync(final String snapshotName) throw new TableNotDisabledException(tableName); } - return internalRestoreSnapshotAsync(snapshotName, tableName, false); + return internalRestoreSnapshotAsync(snapshotName, tableName, false, null); } @Override public Future cloneSnapshotAsync(String snapshotName, TableName tableName, - boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException { + boolean restoreAcl, String customSFT) + throws IOException, TableExistsException, RestoreSnapshotException { if (tableExists(tableName)) { throw new TableExistsException(tableName); } - return internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl); + return internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT); } @Override @@ -2778,7 +2781,7 @@ protected IsProcedureDoneResponse rpcCall() throws Exception { * @throws IllegalArgumentException if the restore request is formatted incorrectly */ private Future internalRestoreSnapshotAsync(final String snapshotName, - final TableName tableName, final boolean restoreAcl) + final TableName tableName, final boolean restoreAcl, String customSFT) throws IOException, RestoreSnapshotException { final SnapshotProtos.SnapshotDescription snapshot = SnapshotProtos.SnapshotDescription.newBuilder() @@ -2793,13 +2796,15 @@ private Future internalRestoreSnapshotAsync(final String snapshotName, Long nonce = ng.newNonce(); @Override protected RestoreSnapshotResponse rpcCall() throws Exception { - final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder() + final RestoreSnapshotRequest.Builder builder = RestoreSnapshotRequest.newBuilder() .setSnapshot(snapshot) .setNonceGroup(nonceGroup) .setNonce(nonce) - .setRestoreACL(restoreAcl) - .build(); - return master.restoreSnapshot(getRpcController(), request); + .setRestoreACL(restoreAcl); + if (customSFT != null) { + builder.setCustomSFT(customSFT); + } + return master.restoreSnapshot(getRpcController(), builder.build()); } }); @@ -4468,5 +4473,4 @@ protected Boolean rpcCall(int callTimeout) throws Exception { } }); } - } 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 a0e5320f855c..64b82cd3c223 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 @@ -1936,7 +1936,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) { @@ -1972,12 +1972,13 @@ 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); @@ -2007,7 +2008,7 @@ private CompletableFuture restoreSnapshot(String snapshotName, TableName t }); return future; } else { - return internalRestoreSnapshot(snapshotName, tableName, restoreAcl); + return internalRestoreSnapshot(snapshotName, tableName, restoreAcl, null); } } @@ -2024,7 +2025,7 @@ private void completeConditionalOnFuture(CompletableFuture dependentFutur @Override public CompletableFuture cloneSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl) { + boolean restoreAcl, String customSFT) { CompletableFuture future = new CompletableFuture<>(); addListener(tableExists(tableName), (exists, err) -> { if (err != null) { @@ -2033,14 +2034,14 @@ public CompletableFuture cloneSnapshot(String snapshotName, TableName tabl future.completeExceptionally(new TableExistsException(tableName)); } else { completeConditionalOnFuture(future, - internalRestoreSnapshot(snapshotName, tableName, restoreAcl)); + internalRestoreSnapshot(snapshotName, tableName, restoreAcl, customSFT)); } }); return future; } private CompletableFuture internalRestoreSnapshot(String snapshotName, TableName tableName, - boolean restoreAcl) { + boolean restoreAcl, String customSFT) { SnapshotProtos.SnapshotDescription snapshot = SnapshotProtos.SnapshotDescription.newBuilder() .setName(snapshotName).setTable(tableName.getNameAsString()).build(); try { @@ -2048,10 +2049,15 @@ 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(customSFT != null){ + builder.setCustomSFT(customSFT); + } 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/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto index 4a6bb3959532..8eccb7a0b084 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto @@ -461,6 +461,7 @@ message RestoreSnapshotRequest { optional uint64 nonce_group = 2 [default = 0]; optional uint64 nonce = 3 [default = 0]; optional bool restoreACL = 4 [default = false]; + optional string customSFT = 5; } message RestoreSnapshotResponse { diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index 66d1640763fc..cdc6e7324965 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -208,6 +208,7 @@ message CloneSnapshotStateData { repeated RegionInfo region_info = 4; repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 5; optional bool restore_acl = 6; + optional string customSFT = 7; } enum RestoreSnapshotState { 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 a13cd0cfcbb4..8fe4324cb58e 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 @@ -2614,8 +2614,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 customSFT) throws IOException { checkInitialized(); getSnapshotManager().checkSnapshotSupport(); @@ -2624,18 +2624,19 @@ public long restoreSnapshot(final SnapshotDescription snapshotDesc, getClusterSchema().getNamespace(dstTable.getNamespaceAsString()); return MasterProcedureUtil.submitProcedure( - new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { - @Override - protected void run() throws IOException { + new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + @Override + protected void run() throws IOException { setProcId( - getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl)); - } + 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/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index a64c990021ad..695c960637ac 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 @@ -1514,7 +1514,7 @@ public RestoreSnapshotResponse restoreSnapshot(RpcController controller, RestoreSnapshotRequest request) throws ServiceException { try { long procId = master.restoreSnapshot(request.getSnapshot(), request.getNonceGroup(), - request.getNonce(), request.getRestoreACL()); + 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 7157fbf04d40..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 @@ -30,6 +30,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; @@ -44,6 +46,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; @@ -72,6 +76,7 @@ public class CloneSnapshotProcedure private TableDescriptor tableDescriptor; private SnapshotDescription snapshot; private boolean restoreAcl; + private String customSFT; private List newRegions = null; private Map > parentsToChildrenPairMap = new HashMap<>(); @@ -95,13 +100,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, restoreAcl, null); + } + public CloneSnapshotProcedure(final MasterProcedureEnv env, final TableDescriptor tableDescriptor, final SnapshotDescription snapshot, - final boolean restoreAcl) { + final boolean restoreAcl, final String customSFT) { super(env); this.tableDescriptor = tableDescriptor; this.snapshot = snapshot; this.restoreAcl = restoreAcl; + this.customSFT = customSFT; getMonitorStatus(); } @@ -139,6 +151,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); @@ -203,6 +216,37 @@ 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(customSFT)){ + return; + } + + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); + builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); + 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(); + } + + private void validateSFT() { + if (StringUtils.isEmpty(customSFT)){ + return; + } + + //if customSFT is invalid getTrackerClass will throw a RuntimeException + Configuration sftConfig = new Configuration(); + sftConfig.set(StoreFileTrackerFactory.TRACKER_IMPL, customSFT); + StoreFileTrackerFactory.getTrackerClass(sftConfig); + } + @Override protected void rollbackState(final MasterProcedureEnv env, final CloneSnapshotState state) throws IOException { @@ -292,6 +336,9 @@ protected void serializeStateData(ProcedureStateSerializer serializer) cloneSnapshotMsg.addParentToChildRegionsPairList(parentToChildrenPair); } } + if (!StringUtils.isEmpty(customSFT)){ + cloneSnapshotMsg.setCustomSFT(customSFT); + } serializer.serialize(cloneSnapshotMsg.build()); } @@ -327,6 +374,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(); } @@ -340,6 +390,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/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java index e9440621e44c..6b28173d4e4c 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 @@ -25,6 +25,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; @@ -89,6 +90,7 @@ public RestoreSnapshotProcedure(final MasterProcedureEnv env, throws HBaseIOException { this(env, tableDescriptor, snapshot, false); } + /** * Constructor * @param env MasterProcedureEnv @@ -386,14 +388,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 ce08164b487a..9196a97fafa3 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 @@ -66,6 +66,7 @@ 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.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessChecker; @@ -751,8 +752,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 customSFT) throws IOException { MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); TableDescriptor htd = TableDescriptorBuilder.copy(tableName, snapshotTableDesc); org.apache.hadoop.hbase.client.SnapshotDescription snapshotPOJO = null; @@ -762,7 +763,7 @@ private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableNam } long procId; try { - procId = cloneSnapshot(snapshot, htd, nonceKey, restoreAcl); + 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); @@ -786,7 +787,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 customSFT) throws HBaseSnapshotException { TableName tableName = tableDescriptor.getTableName(); @@ -803,7 +805,7 @@ synchronized long cloneSnapshot(final SnapshotDescription snapshot, try { long procId = master.getMasterProcedureExecutor().submitProcedure( new CloneSnapshotProcedure(master.getMasterProcedureExecutor().getEnvironment(), - tableDescriptor, snapshot, restoreAcl), + tableDescriptor, snapshot, restoreAcl, customSFT), nonceKey); this.restoreTableToProcIdMap.put(tableName, procId); return procId; @@ -822,7 +824,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 customSFT) throws IOException { FileSystem fs = master.getMasterFileSystem().getFileSystem(); Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir); @@ -854,11 +856,12 @@ 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, + customSFT); } return procId; } @@ -880,6 +883,10 @@ private long restoreSnapshot(final SnapshotDescription reqSnapshot, final TableN 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 + StoreFileTrackerFactory.validatePreRestoreSnapshot(master.getTableDescriptors().get(tableName), + snapshotTableDesc, master.getConfiguration()); + if (master.getTableStateManager().isTableState( TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) { throw new UnsupportedOperationException("Table '" + @@ -921,7 +928,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/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java index 1c683ae3de62..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 @@ -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; @@ -92,7 +93,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; @@ -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 validatePreRestoreSnapshot(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/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java index 0f8a95fc7648..17406445fc3a 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 @@ -57,6 +57,7 @@ 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.security.access.AccessControlClient; @@ -72,9 +73,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; @@ -200,8 +199,8 @@ private RestoreMetaChanges restoreHdfsRegions(final ThreadPoolExecutor exec) thr List tableRegions = getTableRegions(); - RegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor() - .getTableName()); + RegionInfo mobRegion = + MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor().getTableName()); if (tableRegions != null) { // restore the mob region in case if (regionNames.contains(mobRegion.getEncodedName())) { @@ -707,7 +706,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..9497967ab3ce --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java @@ -0,0 +1,72 @@ +/** + * 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 static org.junit.Assert.assertThrows; + +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.Bytes; +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(Bytes.toString(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); + } + + @Test + public void testCloneSnapshotWithIncorrectCustomSFT() throws IOException, InterruptedException { + TableName clonedTableName = + TableName.valueOf(getValidMethodName() + "-" + EnvironmentEdgeManager.currentTime()); + + IOException ioException = assertThrows(IOException.class, () -> { + admin.cloneSnapshot(Bytes.toString(snapshotName1), clonedTableName, false, "IncorrectSFT"); + }); + + assertEquals( + "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 41f2afdfa421..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 @@ -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.validatePreRestoreSnapshot(td, snapTd, conf); + // removing cf level config is fine when it matches the td config + StoreFileTrackerFactory.validatePreRestoreSnapshot(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.validatePreRestoreSnapshot(td, defaultTd, conf); + }); + assertThrows(RestoreSnapshotException.class, () -> { + StoreFileTrackerFactory.validatePreRestoreSnapshot(snapTd, defaultTd, conf); + }); + } } diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 4d80499bb5cf..25f07991a868 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -1237,8 +1237,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 3c637b88029d..76631c91cf9f 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..3edd16d326bf 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 0aa9862c031a..2a5d3370b61b 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 @@ -1065,8 +1065,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 customSFT) throws IOException, TableExistsException, RestoreSnapshotException { throw new NotImplementedException("cloneSnapshotAsync not supported in ThriftAdmin"); } From fed99939a7c47d688e0587d5df9fa89c439de83d Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Thu, 16 Dec 2021 21:07:38 +0000 Subject: [PATCH 16/32] =?UTF-8?q?HBASE-26265=20Update=20ref=20guide=20to?= =?UTF-8?q?=20mention=20the=20new=20store=20file=20tracker=20im=E2=80=A6?= =?UTF-8?q?=20(#3942)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../_chapters/store_file_tracking.adoc | 145 ++++++++++++++++++ src/main/asciidoc/book.adoc | 1 + 2 files changed, 146 insertions(+) create mode 100644 src/main/asciidoc/_chapters/store_file_tracking.adoc diff --git a/src/main/asciidoc/_chapters/store_file_tracking.adoc b/src/main/asciidoc/_chapters/store_file_tracking.adoc new file mode 100644 index 000000000000..74d802f386c5 --- /dev/null +++ b/src/main/asciidoc/_chapters/store_file_tracking.adoc @@ -0,0 +1,145 @@ +//// +/** + * + * 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. + */ +//// + +[[storefiletracking]] += Store File Tracking +:doctype: book +:numbered: +:toc: left +:icons: font +:experimental: + +== Overview + +This feature introduces an abstraction layer to track store files still used/needed by store +engines, allowing for plugging different approaches of identifying store +files required by the given store. + +Historically, HBase internals have relied on creating hfiles on temporary directories first, renaming +those files to the actual store directory at operation commit time. That's a simple and convenient +way to separate transient from already finalised files that are ready to serve client reads with data. +This approach works well with strong consistent file systems, but with the popularity of less consistent +file systems, mainly Object Store which can be used like file systems, dependency on atomic rename operations starts to introduce +performance penalties. The Amazon S3 Object Store, in particular, has been the most affected deployment, +due to its lack of atomic renames. The HBase community temporarily bypassed this problem by building a distributed locking layer called HBOSS, +to guarantee atomicity of operations against S3. + +With *Store File Tracking*, decision on where to originally create new hfiles and how to proceed upon +commit is delegated to the specific Store File Tracking implementation. +The implementation can be set at the HBase service leve in *hbase-site.xml* or at the +Table or Column Family via the TableDescriptor configuration. + +NOTE: When the store file tracking implementation is specified in *hbase_site.xml*, this configuration is also propagated into a tables configuration +at table creation time. This is to avoid dangerous configuration mismatches between processes, which +could potentially lead to data loss. + +== Available Implementations + +Store File Tracking initial version provides three builtin implementations: + +* DEFAULT +* FILE +* MIGRATION + +### DEFAULT + +As per the name, this is the Store File Tracking implementation used by default when no explicit +configuration has been defined. The DEFAULT tracker implements the standard approach using temporary +directories and renames. This is how all previous (implicit) implementation that HBase used to track store files. + +### FILE + +A file tracker implementation that creates new files straight in the store directory, avoiding the +need for rename operations. It keeps a list of committed hfiles in memory, backed by meta files, in +each store directory. Whenever a new hfile is committed, the list of _tracked files_ in the given +store is updated and a new meta file is written with this list contents, discarding the previous +meta file now containing an out dated list. + +### MIGRATION + +A special implementation to be used when swapping between Store File Tracking implementations on +pre-existing tables that already contain data, and therefore, files being tracked under an specific +logic. + +== Usage + +For fresh deployments that don't yet contain any user data, *FILE* implementation can be just set as +value for *hbase.store.file-tracker.impl* property in global *hbase-site.xml* configuration, prior +to the first hbase start. Omitting this property sets the *DEFAULT* implementation. + +For clusters with data that are upgraded to a version of HBase containing the store file tracking +feature, the Store File Tracking implementation can only be changed with the *MIGRATION* +implementation, so that the _new tracker_ can safely build its list of tracked files based on the +list of the _current tracker_. + +NOTE: MIGRATION tracker should NOT be set at global configuration. To use it, follow below section +about setting Store File Tacking at Table or Column Family configuration. + + +### Configuring for Table or Column Family + +Setting Store File Tracking configuration globally may not always be possible or desired, for example, +in the case of upgraded clusters with pre-existing user data. +Store File Tracking can be set at Table or Column Family level configuration. +For example, to specify *FILE* implementation in the table configuration at table creation time, +the following should be applied: + +---- +create 'my-table', 'f1', 'f2', {CONFIGURATION => {'hbase.store.file-tracker.impl' => 'FILE'}} +---- + +To define *FILE* for an specific Column Family: + +---- +create 'my-table', {NAME=> '1', CONFIGURATION => {'hbase.store.file-tracker.impl' => 'FILE'}} +---- + +### Switching trackers at Table or Column Family + +A very common scenario is to set Store File Tracking on pre-existing HBase deployments that have +been upgraded to a version that supports this feature. To apply the FILE tracker, tables effectively +need to be migrated from the DEFAULT tracker to the FILE tracker. As explained previously, such +process requires the usage of the special MIGRATION tracker implementation, which can only be +specified at table or Column Family level. + +For example, to switch _tracker_ from *DEFAULT* to *FILE* in a table configuration: + +---- +alter 'my-table', CONFIGURATION => {'hbase.store.file-tracker.impl' => 'MIGRATION', +'hbase.store.file-tracker.migration.src.impl' => 'DEFAULT', +'hbase.store.file-tracker.migration.dst.impl' => 'FILE'} +---- + +To apply similar switch at column family level configuration: + +---- +alter 'my-table', {NAME => 'f1', CONFIGURATION => {'hbase.store.file-tracker.impl' => 'MIGRATION', +'hbase.store.file-tracker.migration.src.impl' => 'DEFAULT', +'hbase.store.file-tracker.migration.dst.impl' => 'FILE'}} +---- + +Once all table regions have been onlined again, don't forget to disable MIGRATION, by now setting +*hbase.store.file-tracker.migration.dst.impl* value as the *hbase.store.file-tracker.impl*. In the above +example, that would be as follows: + +---- +alter 'my-table', CONFIGURATION => {'hbase.store.file-tracker.impl' => 'FILE'} +---- diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc index c264b06e4c92..24563cdda62c 100644 --- a/src/main/asciidoc/book.adoc +++ b/src/main/asciidoc/book.adoc @@ -87,6 +87,7 @@ include::_chapters/zookeeper.adoc[] include::_chapters/community.adoc[] include::_chapters/hbtop.adoc[] include::_chapters/tracing.adoc[] +include::_chapters/store_file_tracking.adoc[] = Appendix From 44718fae1d13a8a4aed76806352840711d87852e Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Wed, 5 Jan 2022 12:20:07 +0000 Subject: [PATCH 17/32] HBASE-26585 Add SFT configuration to META table descriptor when creating META (#3998) Signed-off-by: Duo Zhang Signed-off-by: Josh Elser (cherry picked from commit baeb51ff8adf1bc3b0309f50af39a9f0df1487ae) --- .../hadoop/hbase/util/FSTableDescriptors.java | 4 +- ...MasterFileSystemWithStoreFileTracking.java | 69 +++++++++++++++++++ 2 files changed, 72 insertions(+), 1 deletion(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 60024001b7db..7570402df271 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -127,7 +128,8 @@ public static TableDescriptor tryUpdateAndGetMetaTableDescriptor(Configuration c return getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME); } catch (TableInfoMissingException e) { TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf); - TableDescriptor td = builder.build(); + TableDescriptor td = StoreFileTrackerFactory. + updateWithTrackerConfigs(conf, builder.build()); LOG.info("Creating new hbase:meta table descriptor {}", td); TableName tableName = td.getTableName(); Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java new file mode 100644 index 000000000000..f2b8a589e85e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java @@ -0,0 +1,69 @@ +/** + * 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; + +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.Trackers.FILE; +import static org.junit.Assert.assertEquals; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +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.junit.rules.TestName; + +/** + * Test the master filesystem in a local cluster with + * Store File Tracking explicitly set in global config + */ +@Category({MasterTests.class, MediumTests.class}) +public class TestMasterFileSystemWithStoreFileTracking { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterFileSystemWithStoreFileTracking.class); + + @Rule + public TestName name = new TestName(); + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void setupTest() throws Exception { + UTIL.getConfiguration().set(TRACKER_IMPL, FILE.name()); + UTIL.startMiniCluster(); + } + + @AfterClass + public static void teardownTest() throws Exception { + UTIL.shutdownMiniCluster(); + } + + @Test + public void tesMetaDescriptorHasSFTConfig() throws Exception { + TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME); + assertEquals(FILE.name(), descriptor.getValue(TRACKER_IMPL)); + } +} From e3861988f67ce6df2c3975f6a328c3d5351b6f55 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 7 Jan 2022 23:05:47 +0800 Subject: [PATCH 18/32] HBASE-26639 The implementation of TestMergesSplitsAddToTracker is problematic (#4010) Signed-off-by: Wellington Ramos Chevreuil Conflicts: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java --- .../TestMergesSplitsAddToTracker.java | 49 +++++++++++-------- .../TestStoreFileTracker.java | 24 ++++++--- 2 files changed, 46 insertions(+), 27 deletions(-) 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 435fa26f7551..2cbfdea94ef7 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 @@ -17,14 +17,12 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory. - TRACKER_IMPL; +import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_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; @@ -36,10 +34,14 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; 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.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.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.regionserver.storefiletracker.TestStoreFileTracker; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -54,7 +56,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; @Category({RegionServerTests.class, LargeTests.class}) @@ -66,14 +67,15 @@ public class TestMergesSplitsAddToTracker { private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - public static final byte[] FAMILY_NAME = Bytes.toBytes("info"); + private static final String FAMILY_NAME_STR = "info"; + + private static final byte[] FAMILY_NAME = Bytes.toBytes(FAMILY_NAME_STR); @Rule - public TestName name = new TestName(); + public TableNameTestRule name = new TableNameTestRule(); @BeforeClass public static void setupClass() throws Exception { - TEST_UTIL.getConfiguration().set(TRACKER_IMPL, TestStoreFileTracker.class.getName()); TEST_UTIL.startMiniCluster(); } @@ -84,13 +86,24 @@ public static void afterClass() throws Exception { @Before public void setup(){ - TestStoreFileTracker.trackedFiles = new HashMap<>(); + TestStoreFileTracker.clear(); + } + + private TableName createTable(byte[] splitKey) throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(name.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_NAME)) + .setValue(TRACKER_IMPL, TestStoreFileTracker.class.getName()).build(); + if (splitKey != null) { + TEST_UTIL.getAdmin().createTable(td, new byte[][] { splitKey }); + } else { + TEST_UTIL.getAdmin().createTable(td); + } + return td.getTableName(); } @Test public void testCommitDaughterRegion() throws Exception { - TableName table = TableName.valueOf(name.getMethodName()); - TEST_UTIL.createTable(table, FAMILY_NAME); + TableName table = createTable(null); //first put some data in order to have a store file created putThreeRowsAndFlush(table); HRegion region = TEST_UTIL.getHBaseCluster().getRegions(table).get(0); @@ -124,8 +137,7 @@ public void testCommitDaughterRegion() throws Exception { @Test public void testCommitMergedRegion() throws Exception { - TableName table = TableName.valueOf(name.getMethodName()); - TEST_UTIL.createTable(table, FAMILY_NAME); + TableName table = createTable(null); //splitting the table first split(table, Bytes.toBytes("002")); //Add data and flush to create files in the two different regions @@ -162,8 +174,7 @@ public void testCommitMergedRegion() throws Exception { @Test public void testSplitLoadsFromTracker() throws Exception { - TableName table = TableName.valueOf(name.getMethodName()); - TEST_UTIL.createTable(table, FAMILY_NAME); + TableName table = createTable(null); //Add data and flush to create files in the two different regions putThreeRowsAndFlush(table); HRegion region = TEST_UTIL.getHBaseCluster().getRegions(table).get(0); @@ -187,9 +198,7 @@ private void split(TableName table, byte[] splitKey) throws IOException { @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")}); + TableName table = createTable(Bytes.toBytes("002")); //Add data and flush to create files in the two different regions putThreeRowsAndFlush(table); List regions = TEST_UTIL.getHBaseCluster().getRegions(table); @@ -237,10 +246,8 @@ private void validateDaughterRegionsFiles(HRegion region, String orignalFileName } 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()); + for (FileStatus f : fs.listStatus(new Path(regionDir, FAMILY_NAME_STR))) { + assertTrue(TestStoreFileTracker.tracked(regionDir.getName(), FAMILY_NAME_STR, f.getPath())); } } 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 b4c70961841b..c89e151b40c6 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,10 +20,13 @@ 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 java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.regionserver.StoreContext; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.slf4j.Logger; @@ -32,7 +35,8 @@ public class TestStoreFileTracker extends DefaultStoreFileTracker { private static final Logger LOG = LoggerFactory.getLogger(TestStoreFileTracker.class); - public static Map> trackedFiles = new HashMap<>(); + private static ConcurrentMap> trackedFiles = + new ConcurrentHashMap<>(); private String storeId; public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { @@ -40,7 +44,7 @@ public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreC 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<>()); + trackedFiles.computeIfAbsent(storeId, v -> new LinkedBlockingQueue<>()); } else { LOG.info("ctx.getRegionFileSystem() returned null. Leaving storeId null."); } @@ -50,11 +54,19 @@ public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreC protected void doAddNewStoreFiles(Collection newFiles) throws IOException { LOG.info("adding to storeId: {}", storeId); trackedFiles.get(storeId).addAll(newFiles); - trackedFiles.putIfAbsent(storeId, (List)newFiles); } @Override public List load() throws IOException { - return trackedFiles.get(storeId); + return new ArrayList<>(trackedFiles.get(storeId)); + } + + public static boolean tracked(String encodedRegionName, String family, Path file) { + BlockingQueue files = trackedFiles.get(encodedRegionName + "-" + family); + return files != null && files.stream().anyMatch(s -> s.getPath().equals(file)); + } + + public static void clear() { + trackedFiles.clear(); } } From 6169def6bc46885b6a9406f00d0431e2d2493243 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 7 Jan 2022 23:39:18 +0800 Subject: [PATCH 19/32] HBASE-26586 Should not rely on the global config when setting SFT implementation for a table while upgrading (#4006) Signed-off-by: GeorryHuang Signed-off-by: Josh Elser Signed-off-by: Wellington Ramos Chevreuil --- .../master/migrate/RollingUpgradeChore.java | 12 +++++------ ... InitializeStoreFileTrackerProcedure.java} | 14 ++++++++----- ...va => TestInitializeStoreFileTracker.java} | 21 ++++++++++++++----- 3 files changed, 31 insertions(+), 16 deletions(-) rename hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/{MigrateStoreFileTrackerProcedure.java => InitializeStoreFileTrackerProcedure.java} (70%) rename hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/{TestMigrateStoreFileTracker.java => TestInitializeStoreFileTracker.java} (83%) 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 index 3896b41f6625..0417b5c8b985 100644 --- 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 @@ -34,7 +34,7 @@ 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.InitializeStoreFileTrackerProcedure; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -60,7 +60,7 @@ public class RollingUpgradeChore extends ScheduledChore { private final static Logger LOG = LoggerFactory.getLogger(RollingUpgradeChore.class); ProcedureExecutor procedureExecutor; private TableDescriptors tableDescriptors; - private List processingProcs = new ArrayList<>(); + private List processingProcs = new ArrayList<>(); public RollingUpgradeChore(MasterServices masterServices) { this(masterServices.getConfiguration(), masterServices.getMasterProcedureExecutor(), @@ -89,9 +89,9 @@ protected void chore() { } private boolean isCompletelyMigrateSFT(int concurrentCount){ - Iterator iter = processingProcs.iterator(); + Iterator iter = processingProcs.iterator(); while(iter.hasNext()){ - MigrateStoreFileTrackerProcedure proc = iter.next(); + InitializeStoreFileTrackerProcedure proc = iter.next(); if(procedureExecutor.isFinished(proc.getProcId())){ iter.remove(); } @@ -120,8 +120,8 @@ private boolean isCompletelyMigrateSFT(int concurrentCount){ for (Map.Entry entry : migrateSFTTables.entrySet()) { TableDescriptor tableDescriptor = entry.getValue(); - MigrateStoreFileTrackerProcedure proc = - new MigrateStoreFileTrackerProcedure(procedureExecutor.getEnvironment(), tableDescriptor); + InitializeStoreFileTrackerProcedure proc = new InitializeStoreFileTrackerProcedure( + procedureExecutor.getEnvironment(), tableDescriptor); procedureExecutor.submitProcedure(proc); processingProcs.add(proc); } 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/InitializeStoreFileTrackerProcedure.java similarity index 70% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/InitializeStoreFileTrackerProcedure.java index 7cf3d1e8b5ac..bd4162c58b27 100644 --- 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/InitializeStoreFileTrackerProcedure.java @@ -19,28 +19,32 @@ import java.util.Optional; import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 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. + * Procedure for setting StoreFileTracker information to table descriptor. */ @InterfaceAudience.Private -public class MigrateStoreFileTrackerProcedure extends ModifyTableDescriptorProcedure { +public class InitializeStoreFileTrackerProcedure extends ModifyTableDescriptorProcedure { - public MigrateStoreFileTrackerProcedure(){} + public InitializeStoreFileTrackerProcedure(){} - public MigrateStoreFileTrackerProcedure(MasterProcedureEnv env, TableDescriptor unmodified) { + public InitializeStoreFileTrackerProcedure(MasterProcedureEnv env, TableDescriptor unmodified) { super(env, unmodified); } @Override protected Optional modify(MasterProcedureEnv env, TableDescriptor current) { if (StringUtils.isEmpty(current.getValue(StoreFileTrackerFactory.TRACKER_IMPL))) { + // no tracker impl means it is a table created in previous version, the tracker impl can only + // be default. TableDescriptor td = - StoreFileTrackerFactory.updateWithTrackerConfigs(env.getMasterConfiguration(), current); + TableDescriptorBuilder.newBuilder(current).setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.DEFAULT.name()).build(); 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/TestInitializeStoreFileTracker.java similarity index 83% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestInitializeStoreFileTracker.java index 4f00184a107f..77719cd74970 100644 --- 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/TestInitializeStoreFileTracker.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase.master.migrate; +import static org.junit.Assert.assertEquals; + import java.io.IOException; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -31,6 +33,7 @@ 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.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; @@ -40,11 +43,11 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category(MediumTests.class) -public class TestMigrateStoreFileTracker { +@Category({ MediumTests.class, MasterTests.class }) +public class TestInitializeStoreFileTracker { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMigrateStoreFileTracker.class); + HBaseClassTestRule.forClass(TestInitializeStoreFileTracker.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); @@ -56,9 +59,12 @@ public class TestMigrateStoreFileTracker { @Before public void setUp() throws Exception { conf = HBaseConfiguration.create(); - //Speed up the launch of RollingUpgradeChore + // 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); + // Set the default implementation to file instead of default, to confirm we will not set SFT to + // file + conf.set(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()); HTU = new HBaseTestingUtility(conf); HTU.startMiniCluster(); } @@ -89,7 +95,7 @@ public void testMigrateStoreFileTracker() throws IOException, InterruptedExcepti HTU.getMiniHBaseCluster().stopMaster(0).join(); HTU.getMiniHBaseCluster().startMaster(); HTU.getMiniHBaseCluster().waitForActiveAndReadyMaster(30000); - //wait until all tables have been migrated + // wait until all tables have been migrated TableDescriptors tds = HTU.getMiniHBaseCluster().getMaster().getTableDescriptors(); HTU.waitFor(30000, () -> { try { @@ -104,5 +110,10 @@ public void testMigrateStoreFileTracker() throws IOException, InterruptedExcepti return false; } }); + for (String table : tables) { + TableDescriptor td = tds.get(TableName.valueOf(table)); + assertEquals(StoreFileTrackerFactory.Trackers.DEFAULT.name(), + td.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + } } } From 3631145727a6ecd6c5c8e91d44059f9f5c673216 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Sun, 16 Jan 2022 16:25:28 +0800 Subject: [PATCH 20/32] HBASE-26654 ModifyTableDescriptorProcedure shoud load TableDescriptor while executing (#4034) Signed-off-by: GeorryHuang --- .../src/main/protobuf/MasterProcedure.proto | 2 +- .../master/migrate/RollingUpgradeChore.java | 2 +- .../ModifyTableDescriptorProcedure.java | 30 ++++++++++++++----- .../InitializeStoreFileTrackerProcedure.java | 5 ++-- 4 files changed, 28 insertions(+), 11 deletions(-) diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index cdc6e7324965..fdcb1818d367 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -593,6 +593,6 @@ enum ModifyTableDescriptorState { } message ModifyTableDescriptorStateData { - required TableSchema unmodified_table_schema = 1; + required TableName table_name = 1; optional TableSchema modified_table_schema = 2; } 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 index 0417b5c8b985..b7087dd352a3 100644 --- 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 @@ -121,7 +121,7 @@ private boolean isCompletelyMigrateSFT(int concurrentCount){ for (Map.Entry entry : migrateSFTTables.entrySet()) { TableDescriptor tableDescriptor = entry.getValue(); InitializeStoreFileTrackerProcedure proc = new InitializeStoreFileTrackerProcedure( - procedureExecutor.getEnvironment(), tableDescriptor); + procedureExecutor.getEnvironment(), tableDescriptor.getTableName()); procedureExecutor.submitProcedure(proc); processingProcs.add(proc); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java index 4f491421854a..0cb83ce55fae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.procedure; import java.io.IOException; +import java.util.Objects; import java.util.Optional; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -44,20 +45,21 @@ public abstract class ModifyTableDescriptorProcedure private static final Logger LOG = LoggerFactory.getLogger(ModifyTableDescriptorProcedure.class); - private TableDescriptor unmodifiedTableDescriptor; + private TableName tableName; + private TableDescriptor modifiedTableDescriptor; protected ModifyTableDescriptorProcedure() { } - protected ModifyTableDescriptorProcedure(MasterProcedureEnv env, TableDescriptor unmodified) { + protected ModifyTableDescriptorProcedure(MasterProcedureEnv env, TableName tableName) { super(env); - this.unmodifiedTableDescriptor = unmodified; + this.tableName = Objects.requireNonNull(tableName); } @Override public TableName getTableName() { - return unmodifiedTableDescriptor.getTableName(); + return tableName; } @Override @@ -82,7 +84,12 @@ protected Flow executeFromState(MasterProcedureEnv env, ModifyTableDescriptorSta try { switch (state) { case MODIFY_TABLE_DESCRIPTOR_PREPARE: - Optional modified = modify(env, unmodifiedTableDescriptor); + TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName); + if (current == null) { + LOG.info("Table {} does not exist, skip modifying", tableName); + return Flow.NO_MORE_STATE; + } + Optional modified = modify(env, current); if (modified.isPresent()) { modifiedTableDescriptor = modified.get(); setNextState(ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_UPDATE); @@ -108,6 +115,15 @@ protected Flow executeFromState(MasterProcedureEnv env, ModifyTableDescriptorSta return Flow.HAS_MORE_STATE; } + @Override + protected boolean holdLock(MasterProcedureEnv env) { + // here we want to make sure that our modification result will not be overwrite by other MTPs, + // so we set holdLock to true. Since we do not need to schedule any sub procedures, especially + // no remote procedures, so it is OK for us a hold the lock all the time, it will not hurt the + // availability too much. + return true; + } + @Override protected void rollbackState(MasterProcedureEnv env, ModifyTableDescriptorState state) throws IOException, InterruptedException { @@ -141,7 +157,7 @@ protected ModifyTableDescriptorState getInitialState() { protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { super.serializeStateData(serializer); ModifyTableDescriptorStateData.Builder builder = ModifyTableDescriptorStateData.newBuilder() - .setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor)); + .setTableName(ProtobufUtil.toProtoTableName(tableName)); if (modifiedTableDescriptor != null) { builder.setModifiedTableSchema(ProtobufUtil.toTableSchema(modifiedTableDescriptor)); } @@ -153,7 +169,7 @@ protected void deserializeStateData(ProcedureStateSerializer serializer) throws super.deserializeStateData(serializer); ModifyTableDescriptorStateData data = serializer.deserialize(ModifyTableDescriptorStateData.class); - unmodifiedTableDescriptor = ProtobufUtil.toTableDescriptor(data.getUnmodifiedTableSchema()); + tableName = ProtobufUtil.toTableName(data.getTableName()); if (data.hasModifiedTableSchema()) { modifiedTableDescriptor = ProtobufUtil.toTableDescriptor(data.getModifiedTableSchema()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/InitializeStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/InitializeStoreFileTrackerProcedure.java index bd4162c58b27..5a88f99588b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/InitializeStoreFileTrackerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/InitializeStoreFileTrackerProcedure.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker; import java.util.Optional; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; @@ -33,8 +34,8 @@ public class InitializeStoreFileTrackerProcedure extends ModifyTableDescriptorPr public InitializeStoreFileTrackerProcedure(){} - public InitializeStoreFileTrackerProcedure(MasterProcedureEnv env, TableDescriptor unmodified) { - super(env, unmodified); + public InitializeStoreFileTrackerProcedure(MasterProcedureEnv env, TableName tableName) { + super(env, tableName); } @Override From 84c833dd51401d55bd182559b25f6c503e06119f Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 19 Jan 2022 13:59:35 +0800 Subject: [PATCH 21/32] HBASE-26674 Should modify filesCompacting under storeWriteLock (#4040) Signed-off-by: Josh Elser --- .../org/apache/hadoop/hbase/regionserver/HStore.java | 9 +++++---- .../apache/hadoop/hbase/regionserver/StoreEngine.java | 6 ++++-- .../org/apache/hadoop/hbase/regionserver/TestHStore.java | 4 ++-- 3 files changed, 11 insertions(+), 8 deletions(-) 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 901615606713..0c214e91da57 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 @@ -1226,13 +1226,14 @@ private void writeCompactionWalRecord(Collection filesCompacted, allowedOnPath = ".*/(HStore|TestHStore).java") void replaceStoreFiles(Collection compactedFiles, Collection result, boolean writeCompactionMarker) throws IOException { - storeEngine.replaceStoreFiles(compactedFiles, result); + storeEngine.replaceStoreFiles(compactedFiles, result, () -> { + synchronized(filesCompacting) { + filesCompacting.removeAll(compactedFiles); + } + }); 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(); 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..d85553ac8082 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 @@ -410,7 +410,8 @@ private void refreshStoreFilesInternal(Collection newFiles) throw List openedFiles = openStoreFiles(toBeAddedFiles, false); // propogate the file changes to the underlying store file manager - replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); // won't throw an exception + replaceStoreFiles(toBeRemovedStoreFiles, openedFiles, () -> { + }); // won't throw an exception } /** @@ -493,12 +494,13 @@ public void addStoreFiles(Collection storeFiles, } public void replaceStoreFiles(Collection compactedFiles, - Collection newFiles) throws IOException { + Collection newFiles, Runnable actionUnderLock) throws IOException { storeFileTracker.replace(StoreUtils.toStoreFileInfo(compactedFiles), StoreUtils.toStoreFileInfo(newFiles)); writeLock(); try { storeFileManager.addCompactionResults(compactedFiles, newFiles); + actionUnderLock.run(); } finally { writeUnlock(); } 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 b4d3b8a02e6a..ea7ff61986b7 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 @@ -1034,14 +1034,14 @@ public void testRefreshStoreFilesNotChanged() throws IOException { // call first time after files changed spiedStoreEngine.refreshStoreFiles(); assertEquals(2, this.store.getStorefilesCount()); - verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any()); + verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any(), any()); // call second time spiedStoreEngine.refreshStoreFiles(); // ensure that replaceStoreFiles is not called, i.e, the times does not change, if files are not // refreshed, - verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any()); + verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any(), any()); } private long countMemStoreScanner(StoreScanner scanner) { From 277e1c8fe54e65bad6df9ac211c78b4d329fa96b Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Mon, 24 Jan 2022 19:45:50 +0800 Subject: [PATCH 22/32] HBASE-26675 Data race on Compactor.writer (#4035) Signed-off-by: Xin Sun --- .../regionserver/compactions/Compactor.java | 20 ++++++++++--------- .../compactions/DefaultCompactor.java | 12 +++++------ 2 files changed, 16 insertions(+), 16 deletions(-) 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 a821a90af50e..d934ecb0c16d 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 @@ -95,7 +95,10 @@ public abstract class Compactor { private final boolean dropCacheMajor; private final boolean dropCacheMinor; - protected T writer = null; + // In compaction process only a single thread will access and write to this field, and + // getCompactionTargets is the only place we will access it other than the compaction thread, so + // make it volatile. + protected volatile T writer = null; //TODO: depending on Store is not good but, realistically, all compactors currently do. Compactor(Configuration conf, HStore store) { @@ -546,17 +549,16 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo, dropDeletesFromRow, dropDeletesToRow); } - public List getCompactionTargets(){ - if (writer == null){ + public List getCompactionTargets() { + T writer = this.writer; + 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()); + if (writer instanceof StoreFileWriter) { + return Arrays.asList(((StoreFileWriter) writer).getPath()); } + return ((AbstractMultiFileWriter) writer).writers().stream().map(sfw -> sfw.getPath()) + .collect(Collectors.toList()); } /** 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 ad2384a97ab8..03e3a1b5f394 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 @@ -74,24 +74,22 @@ protected List commitWriter(FileDetails fd, @Override protected void abortWriter() throws IOException { abortWriter(writer); + // this step signals that the target file is no longer written and can be cleaned up + writer = null; } - protected void abortWriter(StoreFileWriter writer) throws IOException { + protected final 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); } catch (IOException e) { - LOG.warn( - "Failed to delete the leftover file " + leftoverFile + " after an unfinished compaction.", - e); + LOG.warn("Failed to delete the leftover file {} after an unfinished compaction.", + leftoverFile, e); } } } From 31219ba068c4dbbb18d3fed5c399e05cd0e41605 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Tue, 25 Jan 2022 14:51:11 +0800 Subject: [PATCH 23/32] HBASE-26700 The way we bypass broken track file is not enough in StoreFileListFile (#4055) Signed-off-by: Wellington Ramos Chevreuil --- .../storefiletracker/StoreFileListFile.java | 57 ++++-- .../procedure/TestCreateTableProcedure.java | 4 +- .../TestMergesSplitsAddToTracker.java | 9 +- ...cker.java => StoreFileTrackerForTest.java} | 6 +- .../TestStoreFileListFile.java | 165 ++++++++++++++++++ 5 files changed, 216 insertions(+), 25 deletions(-) rename hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/{TestStoreFileTracker.java => StoreFileTrackerForTest.java} (91%) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileListFile.java 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 ffb3647e6259..ced01187b69b 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 @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.regionserver.storefiletracker; +import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; +import java.util.zip.CRC32; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -29,9 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -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; /** @@ -42,18 +41,27 @@ * other file. *

* So in this way, we could avoid listing when we want to load the store file list file. + *

+ * To prevent loading partial file, we use the first 4 bytes as file length, and also append a 4 + * bytes crc32 checksum at the end. This is because the protobuf message parser sometimes can return + * without error on partial bytes if you stop at some special points, but the return message will + * have incorrect field value. We should try our best to prevent this happens because loading an + * incorrect store file list file usually leads to data loss. */ @InterfaceAudience.Private class StoreFileListFile { private static final Logger LOG = LoggerFactory.getLogger(StoreFileListFile.class); - private static final String TRACK_FILE_DIR = ".filelist"; + static final String TRACK_FILE_DIR = ".filelist"; private static final String TRACK_FILE = "f1"; private static final String TRACK_FILE_ROTATE = "f2"; + // 16 MB, which is big enough for a tracker file + private static final int MAX_FILE_SIZE = 16 * 1024 * 1024; + private final StoreContext ctx; private final Path trackFileDir; @@ -74,16 +82,26 @@ class StoreFileListFile { private StoreFileList load(Path path) throws IOException { FileSystem fs = ctx.getRegionFileSystem().getFileSystem(); - byte[] bytes; + byte[] data; + int expectedChecksum; try (FSDataInputStream in = fs.open(path)) { - bytes = ByteStreams.toByteArray(in); + int length = in.readInt(); + if (length <= 0 || length > MAX_FILE_SIZE) { + throw new IOException("Invalid file length " + length + + ", either less than 0 or greater then max allowed size " + MAX_FILE_SIZE); + } + data = new byte[length]; + in.readFully(data); + expectedChecksum = in.readInt(); } - // 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); + CRC32 crc32 = new CRC32(); + crc32.update(data); + int calculatedChecksum = (int) crc32.getValue(); + if (expectedChecksum != calculatedChecksum) { + throw new IOException( + "Checksum mismatch, expected " + expectedChecksum + ", actual " + calculatedChecksum); + } + return StoreFileList.parseFrom(data); } private int select(StoreFileList[] lists) { @@ -101,9 +119,9 @@ StoreFileList load() throws IOException { for (int i = 0; i < 2; i++) { try { lists[i] = load(trackFiles[i]); - } catch (FileNotFoundException | InvalidProtocolBufferException e) { + } catch (FileNotFoundException | EOFException e) { // this is normal case, so use info and do not log stacktrace - LOG.info("Failed to load track file {}: {}", trackFiles[i], e); + LOG.info("Failed to load track file {}: {}", trackFiles[i], e.toString()); } } int winnerIndex = select(lists); @@ -124,10 +142,17 @@ void update(StoreFileList.Builder builder) throws IOException { // 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()); + byte[] actualData = builder.setTimestamp(timestamp).build().toByteArray(); + CRC32 crc32 = new CRC32(); + crc32.update(actualData); + int checksum = (int) crc32.getValue(); + // 4 bytes length at the beginning, plus 4 bytes checksum + FileSystem fs = ctx.getRegionFileSystem().getFileSystem(); try (FSDataOutputStream out = fs.create(trackFiles[nextTrackFile], true)) { - builder.setTimestamp(timestamp).build().writeTo(out); + out.writeInt(actualData.length); + out.write(actualData); + out.writeInt(checksum); } // record timestamp prevTimestamp = timestamp; 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 51ea9f58248a..bb9985e53143 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 @@ -40,7 +40,7 @@ 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.regionserver.storefiletracker.StoreFileTrackerForTest; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -96,7 +96,7 @@ 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(); + String trackerName = StoreFileTrackerForTest.class.getName(); htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACKER_IMPL, trackerName).build(); RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null); long procId = ProcedureTestingUtility.submitAndWait(procExec, 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 2cbfdea94ef7..703d6193e5e5 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 @@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; -import org.apache.hadoop.hbase.regionserver.storefiletracker.TestStoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerForTest; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -86,13 +86,13 @@ public static void afterClass() throws Exception { @Before public void setup(){ - TestStoreFileTracker.clear(); + StoreFileTrackerForTest.clear(); } private TableName createTable(byte[] splitKey) throws IOException { TableDescriptor td = TableDescriptorBuilder.newBuilder(name.getTableName()) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_NAME)) - .setValue(TRACKER_IMPL, TestStoreFileTracker.class.getName()).build(); + .setValue(TRACKER_IMPL, StoreFileTrackerForTest.class.getName()).build(); if (splitKey != null) { TEST_UTIL.getAdmin().createTable(td, new byte[][] { splitKey }); } else { @@ -247,7 +247,8 @@ private void validateDaughterRegionsFiles(HRegion region, String orignalFileName private void verifyFilesAreTracked(Path regionDir, FileSystem fs) throws Exception { for (FileStatus f : fs.listStatus(new Path(regionDir, FAMILY_NAME_STR))) { - assertTrue(TestStoreFileTracker.tracked(regionDir.getName(), FAMILY_NAME_STR, f.getPath())); + assertTrue( + StoreFileTrackerForTest.tracked(regionDir.getName(), FAMILY_NAME_STR, f.getPath())); } } 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/StoreFileTrackerForTest.java similarity index 91% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerForTest.java index c89e151b40c6..abef80acb9d5 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/StoreFileTrackerForTest.java @@ -32,14 +32,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class TestStoreFileTracker extends DefaultStoreFileTracker { +public class StoreFileTrackerForTest extends DefaultStoreFileTracker { - private static final Logger LOG = LoggerFactory.getLogger(TestStoreFileTracker.class); + private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerForTest.class); private static ConcurrentMap> trackedFiles = new ConcurrentHashMap<>(); private String storeId; - public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { + public StoreFileTrackerForTest(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { super(conf, isPrimaryReplica, ctx); if (ctx != null && ctx.getRegionFileSystem() != null) { this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileListFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileListFile.java new file mode 100644 index 000000000000..2aba24b4a46f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileListFile.java @@ -0,0 +1,165 @@ +/** + * 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.assertNull; +import static org.junit.Assert.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.regionserver.StoreContext; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.StoreFileTrackerProtos.StoreFileList; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestStoreFileListFile { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStoreFileListFile.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestStoreFileListFile.class); + + private static final HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility(); + + private Path testDir; + + private StoreFileListFile storeFileListFile; + + @Rule + public TestName name = new TestName(); + + @Before + public void setUp() throws IOException { + testDir = UTIL.getDataTestDir(name.getMethodName()); + HRegionFileSystem hfs = mock(HRegionFileSystem.class); + when(hfs.getFileSystem()).thenReturn(FileSystem.get(UTIL.getConfiguration())); + StoreContext ctx = StoreContext.getBuilder().withFamilyStoreDirectoryPath(testDir) + .withRegionFileSystem(hfs).build(); + storeFileListFile = new StoreFileListFile(ctx); + } + + @AfterClass + public static void tearDown() { + UTIL.cleanupTestDir(); + } + + @Test + public void testEmptyLoad() throws IOException { + assertNull(storeFileListFile.load()); + } + + private FileStatus getOnlyTrackerFile(FileSystem fs) throws IOException { + return fs.listStatus(new Path(testDir, StoreFileListFile.TRACK_FILE_DIR))[0]; + } + + private byte[] readAll(FileSystem fs, Path file) throws IOException { + try (FSDataInputStream in = fs.open(file)) { + return ByteStreams.toByteArray(in); + } + } + + private void write(FileSystem fs, Path file, byte[] buf, int off, int len) throws IOException { + try (FSDataOutputStream out = fs.create(file, true)) { + out.write(buf, off, len); + } + } + + @Test + public void testLoadPartial() throws IOException { + StoreFileList.Builder builder = StoreFileList.newBuilder(); + storeFileListFile.update(builder); + FileSystem fs = FileSystem.get(UTIL.getConfiguration()); + FileStatus trackerFileStatus = getOnlyTrackerFile(fs); + // truncate it so we do not have enough data + LOG.info("Truncate file {} with size {} to {}", trackerFileStatus.getPath(), + trackerFileStatus.getLen(), trackerFileStatus.getLen() / 2); + byte[] content = readAll(fs, trackerFileStatus.getPath()); + write(fs, trackerFileStatus.getPath(), content, 0, content.length / 2); + assertNull(storeFileListFile.load()); + } + + private void writeInt(byte[] buf, int off, int value) { + byte[] b = Bytes.toBytes(value); + for (int i = 0; i < 4; i++) { + buf[off + i] = b[i]; + } + } + + @Test + public void testZeroFileLength() throws IOException { + StoreFileList.Builder builder = StoreFileList.newBuilder(); + storeFileListFile.update(builder); + FileSystem fs = FileSystem.get(UTIL.getConfiguration()); + FileStatus trackerFileStatus = getOnlyTrackerFile(fs); + // write a zero length + byte[] content = readAll(fs, trackerFileStatus.getPath()); + writeInt(content, 0, 0); + write(fs, trackerFileStatus.getPath(), content, 0, content.length); + assertThrows(IOException.class, () -> storeFileListFile.load()); + } + + @Test + public void testBigFileLength() throws IOException { + StoreFileList.Builder builder = StoreFileList.newBuilder(); + storeFileListFile.update(builder); + FileSystem fs = FileSystem.get(UTIL.getConfiguration()); + FileStatus trackerFileStatus = getOnlyTrackerFile(fs); + // write a large length + byte[] content = readAll(fs, trackerFileStatus.getPath()); + writeInt(content, 0, 128 * 1024 * 1024); + write(fs, trackerFileStatus.getPath(), content, 0, content.length); + assertThrows(IOException.class, () -> storeFileListFile.load()); + } + + @Test + public void testChecksumMismatch() throws IOException { + StoreFileList.Builder builder = StoreFileList.newBuilder(); + storeFileListFile.update(builder); + FileSystem fs = FileSystem.get(UTIL.getConfiguration()); + FileStatus trackerFileStatus = getOnlyTrackerFile(fs); + // flip one byte + byte[] content = readAll(fs, trackerFileStatus.getPath()); + content[5] = (byte) ~content[5]; + write(fs, trackerFileStatus.getPath(), content, 0, content.length); + assertThrows(IOException.class, () -> storeFileListFile.load()); + } +} From 922a5166ae128ccfcf3fa499ac361dc4d252d4a1 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 27 Jan 2022 13:28:56 +0800 Subject: [PATCH 24/32] HBASE-26690 Modify FSTableDescriptors to not rely on renaming when writing TableDescriptor (#4054) Signed-off-by: Wellington Ramos Chevreuil --- .../hbase/regionserver/CompactionTool.java | 2 +- .../hadoop/hbase/util/FSTableDescriptors.java | 381 ++++++++---------- .../hbase/util/TestFSTableDescriptors.java | 307 +++++++------- 3 files changed, 317 insertions(+), 373 deletions(-) diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java index 49b9d343c358..fd09e34fde16 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java @@ -191,7 +191,7 @@ private static boolean isRegionDir(final FileSystem fs, final Path path) throws } private static boolean isTableDir(final FileSystem fs, final Path path) throws IOException { - return FSTableDescriptors.getTableInfoPath(fs, path) != null; + return FSTableDescriptors.isTableDir(fs, path); } private static boolean isFamilyDir(final FileSystem fs, final Path path) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 7570402df271..8b854053444f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -17,19 +17,22 @@ */ package org.apache.hadoop.hbase.util; +import com.google.errorprone.annotations.RestrictedApi; import edu.umd.cs.findbugs.annotations.Nullable; +import java.io.EOFException; import java.io.IOException; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -37,7 +40,6 @@ import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableDescriptors; -import org.apache.hadoop.hbase.TableInfoMissingException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -88,8 +90,8 @@ public class FSTableDescriptors implements TableDescriptors { * The file name prefix used to store HTD in HDFS */ static final String TABLEINFO_FILE_PREFIX = ".tableinfo"; - static final String TABLEINFO_DIR = ".tabledesc"; - static final String TMP_DIR = ".tmp"; + + public static final String TABLEINFO_DIR = ".tabledesc"; // This cache does not age out the old stuff. Thinking is that the amount // of data we keep up in here is so small, no need to do occasional purge. @@ -124,22 +126,22 @@ public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOExc public static TableDescriptor tryUpdateAndGetMetaTableDescriptor(Configuration conf, FileSystem fs, Path rootdir) throws IOException { // see if we already have meta descriptor on fs. Write one if not. - try { - return getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME); - } catch (TableInfoMissingException e) { - TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf); - TableDescriptor td = StoreFileTrackerFactory. - updateWithTrackerConfigs(conf, builder.build()); - LOG.info("Creating new hbase:meta table descriptor {}", td); - TableName tableName = td.getTableName(); - Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName); - Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(fs, tableDir, true)); - if (p == null) { - throw new IOException("Failed update hbase:meta table descriptor"); - } - LOG.info("Updated hbase:meta table descriptor to {}", p); - return td; + Optional> opt = getTableDescriptorFromFs(fs, + CommonFSUtils.getTableDir(rootdir, TableName.META_TABLE_NAME), false); + if (opt.isPresent()) { + return opt.get().getSecond(); } + TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf); + TableDescriptor td = StoreFileTrackerFactory.updateWithTrackerConfigs(conf, builder.build()); + LOG.info("Creating new hbase:meta table descriptor {}", td); + TableName tableName = td.getTableName(); + Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName); + Path p = writeTableDescriptor(fs, td, tableDir, null); + if (p == null) { + throw new IOException("Failed update hbase:meta table descriptor"); + } + LOG.info("Updated hbase:meta table descriptor to {}", p); + return td; } public static ColumnFamilyDescriptor getTableFamilyDescForMeta( @@ -220,10 +222,9 @@ public TableDescriptor get(TableName tableName) { } TableDescriptor tdmt = null; try { - tdmt = getTableDescriptorFromFs(fs, rootdir, tableName); - } catch (TableInfoMissingException e) { - // ignore. This is regular operation - } catch (NullPointerException | IOException ioe) { + tdmt = getTableDescriptorFromFs(fs, getTableDir(tableName), fsreadonly).map(Pair::getSecond) + .orElse(null); + } catch (IOException ioe) { LOG.debug("Exception during readTableDecriptor. Current table name = " + tableName, ioe); } // last HTD written wins @@ -297,10 +298,13 @@ public void update(TableDescriptor td, boolean cacheOnly) throws IOException { } } + @RestrictedApi(explanation = "Should only be called in tests or self", link = "", + allowedOnPath = ".*/src/test/.*|.*/FSTableDescriptors\\.java") Path updateTableDescriptor(TableDescriptor td) throws IOException { TableName tableName = td.getTableName(); Path tableDir = getTableDir(tableName); - Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(tableDir)); + Path p = writeTableDescriptor(fs, td, tableDir, + getTableDescriptorFromFs(fs, tableDir, fsreadonly).map(Pair::getFirst).orElse(null)); if (p == null) { throw new IOException("Failed update"); } @@ -328,80 +332,11 @@ public TableDescriptor remove(final TableName tablename) throws IOException { return descriptor; } - private FileStatus getTableInfoPath(Path tableDir) throws IOException { - return getTableInfoPath(fs, tableDir, !fsreadonly); - } - /** - * Find the most current table info file for the table located in the given table directory. - * - * Looks within the {@link #TABLEINFO_DIR} subdirectory of the given directory for any table info - * files and takes the 'current' one - meaning the one with the highest sequence number if present - * or no sequence number at all if none exist (for backward compatibility from before there - * were sequence numbers). - * - * @return The file status of the current table info file or null if it does not exist + * Check whether we have a valid TableDescriptor. */ - public static FileStatus getTableInfoPath(FileSystem fs, Path tableDir) - throws IOException { - return getTableInfoPath(fs, tableDir, false); - } - - /** - * Find the most current table info file for the table in the given table directory. - * - * Looks within the {@link #TABLEINFO_DIR} subdirectory of the given directory for any table info - * files and takes the 'current' one - meaning the one with the highest sequence number if - * present or no sequence number at all if none exist (for backward compatibility from before - * there were sequence numbers). - * If there are multiple table info files found and removeOldFiles is true it also deletes the - * older files. - * - * @return The file status of the current table info file or null if none exist - */ - private static FileStatus getTableInfoPath(FileSystem fs, Path tableDir, boolean removeOldFiles) - throws IOException { - Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR); - return getCurrentTableInfoStatus(fs, tableInfoDir, removeOldFiles); - } - - /** - * Find the most current table info file in the given directory - *

- * Looks within the given directory for any table info files and takes the 'current' one - meaning - * the one with the highest sequence number if present or no sequence number at all if none exist - * (for backward compatibility from before there were sequence numbers). - *

- * If there are multiple possible files found and the we're not in read only mode it also deletes - * the older files. - * @return The file status of the current table info file or null if it does not exist - */ - private static FileStatus getCurrentTableInfoStatus(FileSystem fs, Path dir, - boolean removeOldFiles) throws IOException { - FileStatus[] status = CommonFSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER); - if (status == null || status.length < 1) { - return null; - } - FileStatus mostCurrent = null; - for (FileStatus file : status) { - if (mostCurrent == null || TABLEINFO_FILESTATUS_COMPARATOR.compare(file, mostCurrent) < 0) { - mostCurrent = file; - } - } - if (removeOldFiles && status.length > 1) { - // Clean away old versions - for (FileStatus file : status) { - Path path = file.getPath(); - if (!file.equals(mostCurrent)) { - if (!fs.delete(file.getPath(), false)) { - LOG.warn("Failed cleanup of " + path); - } else { - LOG.debug("Cleaned up old tableinfo file " + path); - } - } - } - } - return mostCurrent; + public static boolean isTableDir(FileSystem fs, Path tableDir) throws IOException { + return getTableDescriptorFromFs(fs, tableDir, true).isPresent(); } /** @@ -411,14 +346,14 @@ private static FileStatus getCurrentTableInfoStatus(FileSystem fs, Path dir, new Comparator() { @Override public int compare(FileStatus left, FileStatus right) { - return right.compareTo(left); + return right.getPath().getName().compareTo(left.getPath().getName()); } }; /** * Return the table directory in HDFS */ - Path getTableDir(final TableName tableName) { + private Path getTableDir(TableName tableName) { return CommonFSUtils.getTableDir(rootdir, tableName); } @@ -449,39 +384,53 @@ private static String formatTableInfoSequenceId(final int number) { return Bytes.toString(b); } - /** - * Regex to eat up sequenceid suffix on a .tableinfo file. - * Use regex because may encounter oldstyle .tableinfos where there is no - * sequenceid on the end. - */ - private static final Pattern TABLEINFO_FILE_REGEX = - Pattern.compile(TABLEINFO_FILE_PREFIX + "(\\.([0-9]{" + WIDTH_OF_SEQUENCE_ID + "}))?$"); + static final class SequenceIdAndFileLength { + + final int sequenceId; + + final int fileLength; + + SequenceIdAndFileLength(int sequenceId, int fileLength) { + this.sequenceId = sequenceId; + this.fileLength = fileLength; + } + } /** + * Returns the current sequence id and file length or 0 if none found. * @param p Path to a .tableinfo file. - * @return The current editid or 0 if none found. */ - static int getTableInfoSequenceId(final Path p) { - if (p == null) { - return 0; + @RestrictedApi(explanation = "Should only be called in tests or self", link = "", + allowedOnPath = ".*/src/test/.*|.*/FSTableDescriptors\\.java") + static SequenceIdAndFileLength getTableInfoSequenceIdAndFileLength(Path p) { + String name = p.getName(); + if (!name.startsWith(TABLEINFO_FILE_PREFIX)) { + throw new IllegalArgumentException("Invalid table descriptor file name: " + name); } - Matcher m = TABLEINFO_FILE_REGEX.matcher(p.getName()); - if (!m.matches()) { - throw new IllegalArgumentException(p.toString()); + int firstDot = name.indexOf('.', TABLEINFO_FILE_PREFIX.length()); + if (firstDot < 0) { + // oldest style where we do not have both sequence id and file length + return new SequenceIdAndFileLength(0, 0); } - String suffix = m.group(2); - if (suffix == null || suffix.length() <= 0) { - return 0; + int secondDot = name.indexOf('.', firstDot + 1); + if (secondDot < 0) { + // old stype where we do not have file length + int sequenceId = Integer.parseInt(name.substring(firstDot + 1)); + return new SequenceIdAndFileLength(sequenceId, 0); } - return Integer.parseInt(m.group(2)); + int sequenceId = Integer.parseInt(name.substring(firstDot + 1, secondDot)); + int fileLength = Integer.parseInt(name.substring(secondDot + 1)); + return new SequenceIdAndFileLength(sequenceId, fileLength); } /** - * @param sequenceid - * @return Name of tableinfo file. + * Returns Name of tableinfo file. */ - static String getTableInfoFileName(final int sequenceid) { - return TABLEINFO_FILE_PREFIX + "." + formatTableInfoSequenceId(sequenceid); + @RestrictedApi(explanation = "Should only be called in tests or self", link = "", + allowedOnPath = ".*/src/test/.*|.*/FSTableDescriptors\\.java") + static String getTableInfoFileName(int sequenceId, byte[] content) { + return TABLEINFO_FILE_PREFIX + "." + formatTableInfoSequenceId(sequenceId) + "." + + content.length; } /** @@ -496,131 +445,135 @@ public static TableDescriptor getTableDescriptorFromFs(FileSystem fs, } /** - * Returns the latest table descriptor for the table located at the given directory - * directly from the file system if it exists. - * @throws TableInfoMissingException if there is no descriptor + * Returns the latest table descriptor for the table located at the given directory directly from + * the file system if it exists. */ public static TableDescriptor getTableDescriptorFromFs(FileSystem fs, Path tableDir) throws IOException { - FileStatus status = getTableInfoPath(fs, tableDir, false); - if (status == null) { - throw new TableInfoMissingException("No table descriptor file under " + tableDir); + return getTableDescriptorFromFs(fs, tableDir, true).map(Pair::getSecond).orElse(null); + } + + private static void deleteMalformedFile(FileSystem fs, Path file) throws IOException { + LOG.info("Delete malformed table descriptor file {}", file); + if (!fs.delete(file, false)) { + LOG.warn("Failed to delete malformed table descriptor file {}", file); } - return readTableDescriptor(fs, status); } - private static TableDescriptor readTableDescriptor(FileSystem fs, FileStatus status) - throws IOException { - int len = Ints.checkedCast(status.getLen()); - byte [] content = new byte[len]; - FSDataInputStream fsDataInputStream = fs.open(status.getPath()); - try { - fsDataInputStream.readFully(content); - } finally { - fsDataInputStream.close(); + private static Optional> getTableDescriptorFromFs(FileSystem fs, + Path tableDir, boolean readonly) throws IOException { + Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR); + FileStatus[] descFiles = CommonFSUtils.listStatus(fs, tableInfoDir, TABLEINFO_PATHFILTER); + if (descFiles == null || descFiles.length < 1) { + return Optional.empty(); } - TableDescriptor htd = null; - try { - htd = TableDescriptorBuilder.parseFrom(content); - } catch (DeserializationException e) { - throw new IOException("content=" + Bytes.toShort(content), e); + Arrays.sort(descFiles, TABLEINFO_FILESTATUS_COMPARATOR); + int i = 0; + TableDescriptor td = null; + FileStatus descFile = null; + for (; i < descFiles.length; i++) { + descFile = descFiles[i]; + Path file = descFile.getPath(); + // get file length from file name if present + int fileLength = getTableInfoSequenceIdAndFileLength(file).fileLength; + byte[] content = new byte[fileLength > 0 ? fileLength : Ints.checkedCast(descFile.getLen())]; + try (FSDataInputStream in = fs.open(file)) { + in.readFully(content); + } catch (EOFException e) { + LOG.info("Failed to load file {} due to EOF, it should be half written: {}", file, + e.toString()); + if (!readonly) { + deleteMalformedFile(fs, file); + } + continue; + } + try { + td = TableDescriptorBuilder.parseFrom(content); + break; + } catch (DeserializationException e) { + LOG.info("Failed to parse file {} due to malformed protobuf message: {}", file, + e.toString()); + if (!readonly) { + deleteMalformedFile(fs, file); + } + } + } + if (!readonly) { + // i + 1 to skip the one we load + for (i = i + 1; i < descFiles.length; i++) { + Path file = descFiles[i].getPath(); + LOG.info("Delete old table descriptor file {}", file); + if (!fs.delete(file, false)) { + LOG.info("Failed to delete old table descriptor file {}", file); + } + } } - return htd; + return td != null ? Optional.of(Pair.newPair(descFile, td)) : Optional.empty(); } /** - * Deletes files matching the table info file pattern within the given directory - * whose sequenceId is at most the given max sequenceId. + * Deletes files matching the table info file pattern within the given directory whose sequenceId + * is at most the given max sequenceId. */ private static void deleteTableDescriptorFiles(FileSystem fs, Path dir, int maxSequenceId) - throws IOException { - FileStatus [] status = CommonFSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER); + throws IOException { + FileStatus[] status = CommonFSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER); for (FileStatus file : status) { Path path = file.getPath(); - int sequenceId = getTableInfoSequenceId(path); + int sequenceId = getTableInfoSequenceIdAndFileLength(path).sequenceId; if (sequenceId <= maxSequenceId) { boolean success = CommonFSUtils.delete(fs, path, false); if (success) { - LOG.debug("Deleted " + path); + LOG.debug("Deleted {}", path); } else { - LOG.error("Failed to delete table descriptor at " + path); + LOG.error("Failed to delete table descriptor at {}", path); } } } } /** - * Attempts to write a new table descriptor to the given table's directory. It first writes it to - * the .tmp dir then uses an atomic rename to move it into place. It begins at the + * Attempts to write a new table descriptor to the given table's directory. It begins at the * currentSequenceId + 1 and tries 10 times to find a new sequence number not already in use. *

* Removes the current descriptor file if passed in. * @return Descriptor file or null if we failed write. */ - private static Path writeTableDescriptor(final FileSystem fs, final TableDescriptor htd, + private static Path writeTableDescriptor(final FileSystem fs, final TableDescriptor td, final Path tableDir, final FileStatus currentDescriptorFile) throws IOException { - // Get temporary dir into which we'll first write a file to avoid half-written file phenomenon. - // This directory is never removed to avoid removing it out from under a concurrent writer. - Path tmpTableDir = new Path(tableDir, TMP_DIR); + // Here we will write to the final directory directly to avoid renaming as on OSS renaming is + // not atomic and has performance issue. The reason why we could do this is that, in the below + // code we will not overwrite existing files, we will write a new file instead. And when + // loading, we will skip the half written file, please see the code in getTableDescriptorFromFs Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR); - // What is current sequenceid? We read the current sequenceid from - // the current file. After we read it, another thread could come in and - // compete with us writing out next version of file. The below retries - // should help in this case some but its hard to do guarantees in face of - // concurrent schema edits. + // In proc v2 we have table lock so typically, there will be no concurrent writes. Keep the + // retry logic here since we may still want to write the table descriptor from for example, + // HBCK2? int currentSequenceId = currentDescriptorFile == null ? 0 : - getTableInfoSequenceId(currentDescriptorFile.getPath()); - int newSequenceId = currentSequenceId; + getTableInfoSequenceIdAndFileLength(currentDescriptorFile.getPath()).sequenceId; // Put arbitrary upperbound on how often we retry - int retries = 10; - int retrymax = currentSequenceId + retries; - Path tableInfoDirPath = null; - do { - newSequenceId += 1; - String filename = getTableInfoFileName(newSequenceId); - Path tempPath = new Path(tmpTableDir, filename); - if (fs.exists(tempPath)) { - LOG.debug(tempPath + " exists; retrying up to " + retries + " times"); + int maxAttempts = 10; + int maxSequenceId = currentSequenceId + maxAttempts; + byte[] bytes = TableDescriptorBuilder.toByteArray(td); + for (int newSequenceId = + currentSequenceId + 1; newSequenceId <= maxSequenceId; newSequenceId++) { + String fileName = getTableInfoFileName(newSequenceId, bytes); + Path filePath = new Path(tableInfoDir, fileName); + try (FSDataOutputStream out = fs.create(filePath, false)) { + out.write(bytes); + } catch (FileAlreadyExistsException e) { + LOG.debug("{} exists; retrying up to {} times", filePath, maxAttempts, e); continue; - } - tableInfoDirPath = new Path(tableInfoDir, filename); - try { - writeTD(fs, tempPath, htd); - fs.mkdirs(tableInfoDirPath.getParent()); - if (!fs.rename(tempPath, tableInfoDirPath)) { - throw new IOException("Failed rename of " + tempPath + " to " + tableInfoDirPath); - } - LOG.debug("Wrote into " + tableInfoDirPath); - } catch (IOException ioe) { - // Presume clash of names or something; go around again. - LOG.debug("Failed write and/or rename; retrying", ioe); - if (!CommonFSUtils.deleteDirectory(fs, tempPath)) { - LOG.warn("Failed cleanup of " + tempPath); - } - tableInfoDirPath = null; + } catch (IOException e) { + LOG.debug("Failed write {}; retrying up to {} times", filePath, maxAttempts, e); continue; } - break; - } while (newSequenceId < retrymax); - if (tableInfoDirPath != null) { - // if we succeeded, remove old table info files. deleteTableDescriptorFiles(fs, tableInfoDir, newSequenceId - 1); + return filePath; } - return tableInfoDirPath; - } - - private static void writeTD(final FileSystem fs, final Path p, final TableDescriptor htd) - throws IOException { - FSDataOutputStream out = fs.create(p, false); - try { - // We used to write this file out as a serialized HTD Writable followed by two '\n's and then - // the toString version of HTD. Now we just write out the pb serialization. - out.write(TableDescriptorBuilder.toByteArray(htd)); - } finally { - out.close(); - } + return null; } /** @@ -678,20 +631,18 @@ public boolean createTableDescriptorForTableDirectory(Path tableDir, TableDescri * @throws IOException if a filesystem error occurs */ public static boolean createTableDescriptorForTableDirectory(FileSystem fs, Path tableDir, - TableDescriptor htd, boolean forceCreation) throws IOException { - FileStatus status = getTableInfoPath(fs, tableDir); - if (status != null) { - LOG.debug("Current path=" + status.getPath()); + TableDescriptor htd, boolean forceCreation) throws IOException { + Optional> opt = getTableDescriptorFromFs(fs, tableDir, false); + if (opt.isPresent()) { + LOG.debug("Current path={}", opt.get().getFirst()); if (!forceCreation) { - if (fs.exists(status.getPath()) && status.getLen() > 0) { - if (readTableDescriptor(fs, status).equals(htd)) { - LOG.trace("TableInfo already exists.. Skipping creation"); - return false; - } + if (htd.equals(opt.get().getSecond())) { + LOG.trace("TableInfo already exists.. Skipping creation"); + return false; } } } - return writeTableDescriptor(fs, htd, tableDir, status) != null; + return writeTableDescriptor(fs, htd, tableDir, opt.map(Pair::getFirst).orElse(null)) != null; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java index c66e88c24bb8..e4eddbba95d0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -28,22 +29,22 @@ import java.util.Arrays; import java.util.Comparator; import java.util.Map; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HBaseCommonTestingUtility; import org.apache.hadoop.hbase.HConstants; 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.exceptions.DeserializationException; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.junit.AfterClass; +import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -63,69 +64,73 @@ public class TestFSTableDescriptors { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestFSTableDescriptors.class); - private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static final HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility(); private static final Logger LOG = LoggerFactory.getLogger(TestFSTableDescriptors.class); @Rule public TestName name = new TestName(); - @Test (expected=IllegalArgumentException.class) + private Path testDir; + + @Before + public void setUp() { + testDir = UTIL.getDataTestDir(name.getMethodName()); + } + + @AfterClass + public static void tearDownAfterClass() { + UTIL.cleanupTestDir(); + } + + @Test(expected = IllegalArgumentException.class) public void testRegexAgainstOldStyleTableInfo() { - Path p = new Path("/tmp", FSTableDescriptors.TABLEINFO_FILE_PREFIX); - int i = FSTableDescriptors.getTableInfoSequenceId(p); + Path p = new Path(testDir, FSTableDescriptors.TABLEINFO_FILE_PREFIX); + int i = FSTableDescriptors.getTableInfoSequenceIdAndFileLength(p).sequenceId; assertEquals(0, i); // Assert it won't eat garbage -- that it fails - p = new Path("/tmp", "abc"); - FSTableDescriptors.getTableInfoSequenceId(p); + p = new Path(testDir, "abc"); + FSTableDescriptors.getTableInfoSequenceIdAndFileLength(p); } @Test public void testCreateAndUpdate() throws IOException { - Path testdir = UTIL.getDataTestDir(name.getMethodName()); TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); FileSystem fs = FileSystem.get(UTIL.getConfiguration()); - FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); + FSTableDescriptors fstd = new FSTableDescriptors(fs, testDir); assertTrue(fstd.createTableDescriptor(htd)); assertFalse(fstd.createTableDescriptor(htd)); - FileStatus[] statuses = fs.listStatus(testdir); - assertTrue("statuses.length=" + statuses.length, statuses.length == 1); + Path tableInfoDir = new Path(CommonFSUtils.getTableDir(testDir, htd.getTableName()), + FSTableDescriptors.TABLEINFO_DIR); + FileStatus[] statuses = fs.listStatus(tableInfoDir); + assertEquals("statuses.length=" + statuses.length, 1, statuses.length); for (int i = 0; i < 10; i++) { fstd.update(htd); } - statuses = fs.listStatus(testdir); - assertTrue(statuses.length == 1); - Path tmpTableDir = new Path(CommonFSUtils.getTableDir(testdir, htd.getTableName()), ".tmp"); - statuses = fs.listStatus(tmpTableDir); - assertTrue(statuses.length == 0); + statuses = fs.listStatus(tableInfoDir); + assertEquals(1, statuses.length); } @Test public void testSequenceIdAdvancesOnTableInfo() throws IOException { - Path testdir = UTIL.getDataTestDir(name.getMethodName()); TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); FileSystem fs = FileSystem.get(UTIL.getConfiguration()); - FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); - Path p0 = fstd.updateTableDescriptor(htd); - int i0 = FSTableDescriptors.getTableInfoSequenceId(p0); - Path p1 = fstd.updateTableDescriptor(htd); - // Assert we cleaned up the old file. - assertTrue(!fs.exists(p0)); - int i1 = FSTableDescriptors.getTableInfoSequenceId(p1); - assertTrue(i1 == i0 + 1); - Path p2 = fstd.updateTableDescriptor(htd); - // Assert we cleaned up the old file. - assertTrue(!fs.exists(p1)); - int i2 = FSTableDescriptors.getTableInfoSequenceId(p2); - assertTrue(i2 == i1 + 1); - Path p3 = fstd.updateTableDescriptor(htd); - // Assert we cleaned up the old file. - assertTrue(!fs.exists(p2)); - int i3 = FSTableDescriptors.getTableInfoSequenceId(p3); - assertTrue(i3 == i2 + 1); - TableDescriptor descriptor = fstd.get(htd.getTableName()); - assertEquals(descriptor, htd); + FSTableDescriptors fstd = new FSTableDescriptors(fs, testDir); + Path previousPath = null; + int previousSeqId = -1; + for (int i = 0; i < 10; i++) { + Path path = fstd.updateTableDescriptor(htd); + int seqId = + FSTableDescriptors.getTableInfoSequenceIdAndFileLength(path).sequenceId; + if (previousPath != null) { + // Assert we cleaned up the old file. + assertTrue(!fs.exists(previousPath)); + assertEquals(previousSeqId + 1, seqId); + } + previousPath = path; + previousSeqId = seqId; + } } @Test @@ -136,7 +141,7 @@ public void testFormatTableInfoSequenceId() { for (int i = 0; i < FSTableDescriptors.WIDTH_OF_SEQUENCE_ID; i++) { sb.append("0"); } - assertEquals(FSTableDescriptors.TABLEINFO_FILE_PREFIX + "." + sb.toString(), + assertEquals(FSTableDescriptors.TABLEINFO_FILE_PREFIX + "." + sb.toString() + ".0", p0.getName()); // Check a few more. Path p2 = assertWriteAndReadSequenceId(2); @@ -154,67 +159,42 @@ public void testFormatTableInfoSequenceId() { } private Path assertWriteAndReadSequenceId(final int i) { - Path p = new Path("/tmp", FSTableDescriptors.getTableInfoFileName(i)); - int ii = FSTableDescriptors.getTableInfoSequenceId(p); + Path p = + new Path(testDir, FSTableDescriptors.getTableInfoFileName(i, HConstants.EMPTY_BYTE_ARRAY)); + int ii = FSTableDescriptors.getTableInfoSequenceIdAndFileLength(p).sequenceId; assertEquals(i, ii); return p; } @Test public void testRemoves() throws IOException { - final String name = this.name.getMethodName(); FileSystem fs = FileSystem.get(UTIL.getConfiguration()); // Cleanup old tests if any detrius laying around. - Path rootdir = new Path(UTIL.getDataTestDir(), name); - TableDescriptors htds = new FSTableDescriptors(fs, rootdir); - TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build(); + TableDescriptors htds = new FSTableDescriptors(fs, testDir); + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); htds.update(htd); assertNotNull(htds.remove(htd.getTableName())); assertNull(htds.remove(htd.getTableName())); } - @Test public void testReadingHTDFromFS() throws IOException { - final String name = this.name.getMethodName(); + @Test + public void testReadingHTDFromFS() throws IOException { FileSystem fs = FileSystem.get(UTIL.getConfiguration()); - TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build(); - Path rootdir = UTIL.getDataTestDir(name); - FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir); + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); + FSTableDescriptors fstd = new FSTableDescriptors(fs, testDir); fstd.createTableDescriptor(htd); TableDescriptor td2 = - FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName()); + FSTableDescriptors.getTableDescriptorFromFs(fs, testDir, htd.getTableName()); assertTrue(htd.equals(td2)); } - @Test public void testReadingOldHTDFromFS() throws IOException, DeserializationException { - final String name = this.name.getMethodName(); - FileSystem fs = FileSystem.get(UTIL.getConfiguration()); - Path rootdir = UTIL.getDataTestDir(name); - FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir); - TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build(); - Path descriptorFile = fstd.updateTableDescriptor(htd); - try (FSDataOutputStream out = fs.create(descriptorFile, true)) { - out.write(TableDescriptorBuilder.toByteArray(htd)); - } - FSTableDescriptors fstd2 = new FSTableDescriptors(fs, rootdir); - TableDescriptor td2 = fstd2.get(htd.getTableName()); - assertEquals(htd, td2); - FileStatus descriptorFile2 = - FSTableDescriptors.getTableInfoPath(fs, fstd2.getTableDir(htd.getTableName())); - byte[] buffer = TableDescriptorBuilder.toByteArray(htd); - try (FSDataInputStream in = fs.open(descriptorFile2.getPath())) { - in.readFully(buffer); - } - TableDescriptor td3 = TableDescriptorBuilder.parseFrom(buffer); - assertEquals(htd, td3); - } - - @Test public void testTableDescriptors() - throws IOException, InterruptedException { - final String name = this.name.getMethodName(); + @Test + public void testTableDescriptors() throws IOException, InterruptedException { FileSystem fs = FileSystem.get(UTIL.getConfiguration()); // Cleanup old tests if any debris laying around. - Path rootdir = new Path(UTIL.getDataTestDir(), name); - FSTableDescriptors htds = new FSTableDescriptors(fs, rootdir) { + FSTableDescriptors htds = new FSTableDescriptors(fs, testDir) { @Override public TableDescriptor get(TableName tablename) { LOG.info(tablename + ", cachehits=" + this.cachehits); @@ -224,28 +204,30 @@ public TableDescriptor get(TableName tablename) { final int count = 10; // Write out table infos. for (int i = 0; i < count; i++) { - htds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); + htds.createTableDescriptor( + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName() + i)).build()); } for (int i = 0; i < count; i++) { - assertTrue(htds.get(TableName.valueOf(name + i)) != null); + assertTrue(htds.get(TableName.valueOf(name.getMethodName() + i)) != null); } for (int i = 0; i < count; i++) { - assertTrue(htds.get(TableName.valueOf(name + i)) != null); + assertTrue(htds.get(TableName.valueOf(name.getMethodName() + i)) != null); } // Update the table infos for (int i = 0; i < count; i++) { - TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)); + TableDescriptorBuilder builder = + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName() + i)); builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of("" + i)); htds.update(builder.build()); } // Wait a while so mod time we write is for sure different. Thread.sleep(100); for (int i = 0; i < count; i++) { - assertTrue(htds.get(TableName.valueOf(name + i)) != null); + assertTrue(htds.get(TableName.valueOf(name.getMethodName() + i)) != null); } for (int i = 0; i < count; i++) { - assertTrue(htds.get(TableName.valueOf(name + i)) != null); + assertTrue(htds.get(TableName.valueOf(name.getMethodName() + i)) != null); } assertEquals(count * 4, htds.invocations); assertTrue("expected=" + (count * 2) + ", actual=" + htds.cachehits, @@ -253,109 +235,103 @@ public TableDescriptor get(TableName tablename) { } @Test - public void testTableDescriptorsNoCache() - throws IOException, InterruptedException { - final String name = this.name.getMethodName(); + public void testTableDescriptorsNoCache() throws IOException, InterruptedException { FileSystem fs = FileSystem.get(UTIL.getConfiguration()); // Cleanup old tests if any debris laying around. - Path rootdir = new Path(UTIL.getDataTestDir(), name); - FSTableDescriptors htds = new FSTableDescriptorsTest(fs, rootdir, false); + FSTableDescriptors htds = new FSTableDescriptorsTest(fs, testDir, false); final int count = 10; // Write out table infos. for (int i = 0; i < count; i++) { - htds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); + htds.createTableDescriptor( + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName() + i)).build()); } for (int i = 0; i < 2 * count; i++) { - assertNotNull("Expected HTD, got null instead", htds.get(TableName.valueOf(name + i % 2))); + assertNotNull("Expected HTD, got null instead", + htds.get(TableName.valueOf(name.getMethodName() + i % 2))); } // Update the table infos for (int i = 0; i < count; i++) { - TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)); + TableDescriptorBuilder builder = + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName() + i)); builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of("" + i)); htds.update(builder.build()); } for (int i = 0; i < count; i++) { - assertNotNull("Expected HTD, got null instead", htds.get(TableName.valueOf(name + i))); - assertTrue("Column Family " + i + " missing", - htds.get(TableName.valueOf(name + i)).hasColumnFamily(Bytes.toBytes("" + i))); + assertNotNull("Expected HTD, got null instead", + htds.get(TableName.valueOf(name.getMethodName() + i))); + assertTrue("Column Family " + i + " missing", htds + .get(TableName.valueOf(name.getMethodName() + i)).hasColumnFamily(Bytes.toBytes("" + i))); } assertEquals(count * 4, htds.invocations); assertEquals("expected=0, actual=" + htds.cachehits, 0, htds.cachehits); } @Test - public void testGetAll() - throws IOException, InterruptedException { + public void testGetAll() throws IOException, InterruptedException { final String name = "testGetAll"; FileSystem fs = FileSystem.get(UTIL.getConfiguration()); // Cleanup old tests if any debris laying around. - Path rootdir = new Path(UTIL.getDataTestDir(), name); - FSTableDescriptors htds = new FSTableDescriptorsTest(fs, rootdir); + FSTableDescriptors htds = new FSTableDescriptorsTest(fs, testDir); final int count = 4; // Write out table infos. for (int i = 0; i < count; i++) { - htds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); + htds.createTableDescriptor( + TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); } // add hbase:meta - htds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build()); - - assertEquals("getAll() didn't return all TableDescriptors, expected: " + - (count + 1) + " got: " + htds.getAll().size(), - count + 1, htds.getAll().size()); + htds + .createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build()); + assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + + " got: " + htds.getAll().size(), count + 1, htds.getAll().size()); } @Test public void testGetAllOrdering() throws Exception { - final String name = "testGetAllOrdering"; FileSystem fs = FileSystem.get(UTIL.getConfiguration()); - Path rootDir = new Path(UTIL.getDataTestDir(), name); - FSTableDescriptors tds = new FSTableDescriptorsTest(fs, rootDir); + FSTableDescriptors tds = new FSTableDescriptorsTest(fs, testDir); String[] tableNames = new String[] { "foo", "bar", "foo:bar", "bar:foo" }; for (String tableName : tableNames) { tds.createTableDescriptor( - TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)).build()); + TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)).build()); } Map tables = tds.getAll(); - // Remove hbase:meta from list. It shows up now since we made it dynamic. The schema + // Remove hbase:meta from list. It shows up now since we made it dynamic. The schema // is written into the fs by the FSTableDescriptors constructor now where before it // didn't. tables.remove(TableName.META_TABLE_NAME.getNameAsString()); assertEquals(4, tables.size()); - String[] tableNamesOrdered = - new String[] { "bar:foo", "default:bar", "default:foo", "foo:bar" }; + new String[] { "bar:foo", "default:bar", "default:foo", "foo:bar" }; int i = 0; for (Map.Entry entry : tables.entrySet()) { assertEquals(tableNamesOrdered[i], entry.getKey()); assertEquals(tableNamesOrdered[i], - entry.getValue().getTableName().getNameWithNamespaceInclAsString()); + entry.getValue().getTableName().getNameWithNamespaceInclAsString()); i++; } } @Test - public void testCacheConsistency() - throws IOException, InterruptedException { - final String name = this.name.getMethodName(); + public void testCacheConsistency() throws IOException, InterruptedException { FileSystem fs = FileSystem.get(UTIL.getConfiguration()); // Cleanup old tests if any debris laying around. - Path rootdir = new Path(UTIL.getDataTestDir(), name); - FSTableDescriptors chtds = new FSTableDescriptorsTest(fs, rootdir); - FSTableDescriptors nonchtds = new FSTableDescriptorsTest(fs, rootdir, false); + FSTableDescriptors chtds = new FSTableDescriptorsTest(fs, testDir); + FSTableDescriptors nonchtds = new FSTableDescriptorsTest(fs, testDir, false); final int count = 10; // Write out table infos via non-cached FSTableDescriptors for (int i = 0; i < count; i++) { - nonchtds.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); + nonchtds.createTableDescriptor( + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName() + i)).build()); } // Calls to getAll() won't increase the cache counter, do per table. for (int i = 0; i < count; i++) { - assertTrue(chtds.get(TableName.valueOf(name + i)) != null); + assertTrue(chtds.get(TableName.valueOf(name.getMethodName() + i)) != null); } assertTrue(nonchtds.getAll().size() == chtds.getAll().size()); @@ -384,23 +360,20 @@ public void testCacheConsistency() @Test public void testNoSuchTable() throws IOException { - final String name = "testNoSuchTable"; FileSystem fs = FileSystem.get(UTIL.getConfiguration()); // Cleanup old tests if any detrius laying around. - Path rootdir = new Path(UTIL.getDataTestDir(), name); - TableDescriptors htds = new FSTableDescriptors(fs, rootdir); + TableDescriptors htds = new FSTableDescriptors(fs, testDir); assertNull("There shouldn't be any HTD for this table", htds.get(TableName.valueOf("NoSuchTable"))); } @Test public void testUpdates() throws IOException { - final String name = "testUpdates"; FileSystem fs = FileSystem.get(UTIL.getConfiguration()); // Cleanup old tests if any detrius laying around. - Path rootdir = new Path(UTIL.getDataTestDir(), name); - TableDescriptors htds = new FSTableDescriptors(fs, rootdir); - TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build(); + TableDescriptors htds = new FSTableDescriptors(fs, testDir); + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); htds.update(htd); htds.update(htd); htds.update(htd); @@ -408,18 +381,15 @@ public void testUpdates() throws IOException { @Test public void testTableInfoFileStatusComparator() { - FileStatus bare = - new FileStatus(0, false, 0, 0, -1, - new Path("/tmp", FSTableDescriptors.TABLEINFO_FILE_PREFIX)); - FileStatus future = - new FileStatus(0, false, 0, 0, -1, - new Path("/tmp/tablinfo." + EnvironmentEdgeManager.currentTime())); - FileStatus farFuture = - new FileStatus(0, false, 0, 0, -1, - new Path("/tmp/tablinfo." + EnvironmentEdgeManager.currentTime() + 1000)); - FileStatus [] alist = {bare, future, farFuture}; - FileStatus [] blist = {bare, farFuture, future}; - FileStatus [] clist = {farFuture, bare, future}; + FileStatus bare = new FileStatus(0, false, 0, 0, -1, + new Path("/tmp", FSTableDescriptors.TABLEINFO_FILE_PREFIX)); + FileStatus future = new FileStatus(0, false, 0, 0, -1, + new Path("/tmp/tablinfo." + EnvironmentEdgeManager.currentTime())); + FileStatus farFuture = new FileStatus(0, false, 0, 0, -1, + new Path("/tmp/tablinfo." + EnvironmentEdgeManager.currentTime() + 1000)); + FileStatus[] alist = { bare, future, farFuture }; + FileStatus[] blist = { bare, farFuture, future }; + FileStatus[] clist = { farFuture, bare, future }; Comparator c = FSTableDescriptors.TABLEINFO_FILESTATUS_COMPARATOR; Arrays.sort(alist, c); Arrays.sort(blist, c); @@ -428,7 +398,7 @@ public void testTableInfoFileStatusComparator() { for (int i = 0; i < alist.length; i++) { assertTrue(alist[i].equals(blist[i])); assertTrue(blist[i].equals(clist[i])); - assertTrue(clist[i].equals(i == 0? farFuture: i == 1? future: bare)); + assertTrue(clist[i].equals(i == 0 ? farFuture : i == 1 ? future : bare)); } } @@ -437,34 +407,57 @@ public void testReadingInvalidDirectoryFromFS() throws IOException { FileSystem fs = FileSystem.get(UTIL.getConfiguration()); try { new FSTableDescriptors(fs, CommonFSUtils.getRootDir(UTIL.getConfiguration())) - .get(TableName.valueOf(HConstants.HBASE_TEMP_DIRECTORY)); + .get(TableName.valueOf(HConstants.HBASE_TEMP_DIRECTORY)); fail("Shouldn't be able to read a table descriptor for the archive directory."); } catch (Exception e) { - LOG.debug("Correctly got error when reading a table descriptor from the archive directory: " - + e.getMessage()); + LOG.debug("Correctly got error when reading a table descriptor from the archive directory: " + + e.getMessage()); } } @Test public void testCreateTableDescriptorUpdatesIfExistsAlready() throws IOException { - Path testdir = UTIL.getDataTestDir(name.getMethodName()); - TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); + TableDescriptor htd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build(); FileSystem fs = FileSystem.get(UTIL.getConfiguration()); - FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir); + FSTableDescriptors fstd = new FSTableDescriptors(fs, testDir); assertTrue(fstd.createTableDescriptor(htd)); assertFalse(fstd.createTableDescriptor(htd)); htd = TableDescriptorBuilder.newBuilder(htd) - .setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue")) - .build(); - assertTrue(fstd.createTableDescriptor(htd)); //this will re-create - Path tableDir = fstd.getTableDir(htd.getTableName()); - Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR); - FileStatus[] statuses = fs.listStatus(tmpTableDir); - assertTrue(statuses.length == 0); - + .setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue")).build(); + assertTrue(fstd.createTableDescriptor(htd)); // this will re-create + Path tableDir = CommonFSUtils.getTableDir(testDir, htd.getTableName()); assertEquals(htd, FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir)); } + @Test + public void testIgnoreBrokenTableDescriptorFiles() throws IOException { + TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf")).build(); + TableDescriptor newHtd = + TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf2")).build(); + assertNotEquals(newHtd, htd); + FileSystem fs = FileSystem.get(UTIL.getConfiguration()); + FSTableDescriptors fstd = new FSTableDescriptors(fs, testDir, false, false); + fstd.update(htd); + byte[] bytes = TableDescriptorBuilder.toByteArray(newHtd); + Path tableDir = CommonFSUtils.getTableDir(testDir, htd.getTableName()); + Path tableInfoDir = new Path(tableDir, FSTableDescriptors.TABLEINFO_DIR); + FileStatus[] statuses = fs.listStatus(tableInfoDir); + assertEquals(1, statuses.length); + int seqId = + FSTableDescriptors.getTableInfoSequenceIdAndFileLength(statuses[0].getPath()).sequenceId + 1; + Path brokenFile = new Path(tableInfoDir, FSTableDescriptors.getTableInfoFileName(seqId, bytes)); + try (FSDataOutputStream out = fs.create(brokenFile)) { + out.write(bytes, 0, bytes.length / 2); + } + assertTrue(fs.exists(brokenFile)); + TableDescriptor getTd = fstd.get(htd.getTableName()); + assertEquals(htd, getTd); + assertFalse(fs.exists(brokenFile)); + } + private static class FSTableDescriptorsTest extends FSTableDescriptors { public FSTableDescriptorsTest(FileSystem fs, Path rootdir) { From 068ab828d419bf6254182fb5c00463a6a0ca2781 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Sun, 30 Jan 2022 20:43:23 +0800 Subject: [PATCH 25/32] HBASE-26587 Introduce a new Admin API to change SFT implementation (#4030) (#4080) Signed-off-by: Wellington Ramos Chevreuil Reviewed-by: Josh Elser --- .../org/apache/hadoop/hbase/client/Admin.java | 48 +++- .../hadoop/hbase/client/AsyncAdmin.java | 16 ++ .../hadoop/hbase/client/AsyncHBaseAdmin.java | 11 + .../client/ConnectionImplementation.java | 18 ++ .../hadoop/hbase/client/HBaseAdmin.java | 109 +++++-- .../hbase/client/RawAsyncHBaseAdmin.java | 91 ++++-- .../client/ShortCircuitMasterConnection.java | 16 ++ .../shaded/protobuf/RequestConverter.java | 28 +- .../src/main/protobuf/Master.proto | 29 ++ .../src/main/protobuf/MasterProcedure.proto | 15 + .../hbase/coprocessor/MasterObserver.java | 52 ++++ .../apache/hadoop/hbase/master/HMaster.java | 54 ++++ .../hbase/master/MasterCoprocessorHost.java | 49 ++++ .../hbase/master/MasterRpcServices.java | 30 ++ .../hadoop/hbase/master/MasterServices.java | 19 ++ .../procedure/CreateTableProcedure.java | 4 +- .../procedure/ModifyTableProcedure.java | 4 +- .../master/snapshot/SnapshotManager.java | 6 +- ...ColumnFamilyStoreFileTrackerProcedure.java | 107 +++++++ .../ModifyStoreFileTrackerProcedure.java | 271 ++++++++++++++++++ .../ModifyTableStoreFileTrackerProcedure.java | 71 +++++ .../StoreFileTrackerFactory.java | 192 +------------ .../StoreFileTrackerValidationUtils.java | 212 ++++++++++++++ .../security/access/AccessController.java | 20 +- .../hadoop/hbase/client/TestAdmin3.java | 157 ++++++++++ .../TestAsyncAdminModifyStoreFileTracker.java | 197 +++++++++++++ .../hbase/master/MockNoopMasterServices.java | 12 + .../TestStoreFileTrackerFactory.java | 52 ---- .../TestStoreFileTrackerValidationUtils.java | 88 ++++++ .../hbase/thrift2/client/ThriftAdmin.java | 14 + 30 files changed, 1716 insertions(+), 276 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyColumnFamilyStoreFileTrackerProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyStoreFileTrackerProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyTableStoreFileTrackerProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminModifyStoreFileTracker.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerValidationUtils.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 36ebc9e3b391..0be8173dcdd5 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 @@ -70,7 +70,6 @@ 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 @@ -806,6 +805,31 @@ default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor colu Future modifyColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException; + /** + * Change the store file tracker of the given table's given family. + * @param tableName the table you want to change + * @param family the family you want to change + * @param dstSFT the destination store file tracker + * @throws IOException if a remote or network exception occurs + */ + default void modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT) + throws IOException { + get(modifyColumnFamilyStoreFileTrackerAsync(tableName, family, dstSFT), getSyncWaitTimeout(), + TimeUnit.MILLISECONDS); + } + + /** + * Change the store file tracker of the given table's given family. + * @param tableName the table you want to change + * @param family the family you want to change + * @param dstSFT the destination store file tracker + * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the + * operation to complete + * @throws IOException if a remote or network exception occurs + */ + Future modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family, + String dstSFT) throws IOException; + /** * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins. * @@ -1633,6 +1657,28 @@ default Future modifyTableAsync(TableName tableName, TableDescriptor td) */ Future modifyTableAsync(TableDescriptor td) throws IOException; + /** + * Change the store file tracker of the given table. + * @param tableName the table you want to change + * @param dstSFT the destination store file tracker + * @throws IOException if a remote or network exception occurs + */ + default void modifyTableStoreFileTracker(TableName tableName, String dstSFT) throws IOException { + get(modifyTableStoreFileTrackerAsync(tableName, dstSFT), getSyncWaitTimeout(), + TimeUnit.MILLISECONDS); + } + + /** + * Change the store file tracker of the given table. + * @param tableName the table you want to change + * @param dstSFT the destination store file tracker + * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the + * operation to complete + * @throws IOException if a remote or network exception occurs + */ + Future modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT) + throws IOException; + /** * Shuts down the HBase cluster. *

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 42c3f049867e..62c9e32f8dc5 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 @@ -178,6 +178,13 @@ CompletableFuture createTable(TableDescriptor desc, byte[] startKey, byte[ */ CompletableFuture modifyTable(TableDescriptor desc); + /** + * Change the store file tracker of the given table. + * @param tableName the table you want to change + * @param dstSFT the destination store file tracker + */ + CompletableFuture modifyTableStoreFileTracker(TableName tableName, String dstSFT); + /** * Deletes a table. * @param tableName name of table to delete @@ -259,6 +266,15 @@ CompletableFuture addColumnFamily(TableName tableName, CompletableFuture modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily); + /** + * Change the store file tracker of the given table's given family. + * @param tableName the table you want to change + * @param family the family you want to change + * @param dstSFT the destination store file tracker + */ + CompletableFuture modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, + String dstSFT); + /** * Create a new namespace. * @param descriptor descriptor which describes the new namespace 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 16ebd1587b47..e2f8094bd75d 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 @@ -142,6 +142,11 @@ public CompletableFuture modifyTable(TableDescriptor desc) { return wrap(rawAdmin.modifyTable(desc)); } + @Override + public CompletableFuture modifyTableStoreFileTracker(TableName tableName, String dstSFT) { + return wrap(rawAdmin.modifyTableStoreFileTracker(tableName, dstSFT)); + } + @Override public CompletableFuture deleteTable(TableName tableName) { return wrap(rawAdmin.deleteTable(tableName)); @@ -199,6 +204,12 @@ public CompletableFuture modifyColumnFamily(TableName tableName, return wrap(rawAdmin.modifyColumnFamily(tableName, columnFamily)); } + @Override + public CompletableFuture modifyColumnFamilyStoreFileTracker(TableName tableName, + byte[] family, String dstSFT) { + return wrap(rawAdmin.modifyColumnFamilyStoreFileTracker(tableName, family, dstSFT)); + } + @Override public CompletableFuture createNamespace(NamespaceDescriptor descriptor) { return wrap(rawAdmin.createNamespace(descriptor)); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 04ca5ee9ed53..13715d9471ac 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -124,6 +124,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest; @@ -1948,6 +1952,20 @@ public HBaseProtos.LogEntry getLogEntries(RpcController controller, HBaseProtos.LogRequest request) throws ServiceException { return stub.getLogEntries(controller, request); } + + @Override + public ModifyTableStoreFileTrackerResponse modifyTableStoreFileTracker( + RpcController controller, ModifyTableStoreFileTrackerRequest request) + throws ServiceException { + return stub.modifyTableStoreFileTracker(controller, request); + } + + @Override + public ModifyColumnStoreFileTrackerResponse modifyColumnStoreFileTracker( + RpcController controller, ModifyColumnStoreFileTrackerRequest request) + throws ServiceException { + return stub.modifyColumnStoreFileTracker(controller, request); + } }; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index b860bdc38cd2..b87874e1994a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -198,10 +198,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse; @@ -368,19 +372,54 @@ public TableDescriptor getDescriptor(TableName tableName) public Future modifyTableAsync(TableDescriptor td) throws IOException { ModifyTableResponse response = executeCallable( new MasterCallable(getConnection(), getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); + long nonceGroup = ng.getNonceGroup(); + long nonce = ng.newNonce(); + @Override protected ModifyTableResponse rpcCall() throws Exception { setPriority(td.getTableName()); - ModifyTableRequest request = RequestConverter.buildModifyTableRequest( - td.getTableName(), td, nonceGroup, nonce); + ModifyTableRequest request = + RequestConverter.buildModifyTableRequest(td.getTableName(), td, nonceGroup, nonce); return master.modifyTable(getRpcController(), request); } }); return new ModifyTableFuture(this, td.getTableName(), response); } + + @Override + public Future modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT) + throws IOException { + ModifyTableStoreFileTrackerResponse response = + executeCallable(new MasterCallable(getConnection(), + getRpcControllerFactory()) { + long nonceGroup = ng.getNonceGroup(); + long nonce = ng.newNonce(); + + @Override + protected ModifyTableStoreFileTrackerResponse rpcCall() throws Exception { + setPriority(tableName); + ModifyTableStoreFileTrackerRequest request = RequestConverter + .buildModifyTableStoreFileTrackerRequest(tableName, dstSFT, nonceGroup, nonce); + return master.modifyTableStoreFileTracker(getRpcController(), request); + } + }); + return new ModifyTablerStoreFileTrackerFuture(this, tableName, response); + } + + private static class ModifyTablerStoreFileTrackerFuture extends ModifyTableFuture { + public ModifyTablerStoreFileTrackerFuture(HBaseAdmin admin, TableName tableName, + ModifyTableStoreFileTrackerResponse response) { + super(admin, tableName, + (response != null && response.hasProcId()) ? response.getProcId() : null); + } + + @Override + public String getOperationType() { + return "MODIFY_TABLE_STORE_FILE_TRACKER"; + } + } + @Override public List listTableDescriptorsByNamespace(byte[] name) throws IOException { return executeCallable(new MasterCallable>(getConnection(), @@ -1077,21 +1116,20 @@ public String getOperationType() { @Override public Future modifyColumnFamilyAsync(final TableName tableName, - final ColumnFamilyDescriptor columnFamily) throws IOException { - ModifyColumnResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected ModifyColumnResponse rpcCall() throws Exception { - setPriority(tableName); - ModifyColumnRequest req = - RequestConverter.buildModifyColumnRequest(tableName, columnFamily, - nonceGroup, nonce); - return master.modifyColumn(getRpcController(), req); - } - }); + final ColumnFamilyDescriptor columnFamily) throws IOException { + ModifyColumnResponse response = executeCallable( + new MasterCallable(getConnection(), getRpcControllerFactory()) { + long nonceGroup = ng.getNonceGroup(); + long nonce = ng.newNonce(); + + @Override + protected ModifyColumnResponse rpcCall() throws Exception { + setPriority(tableName); + ModifyColumnRequest req = + RequestConverter.buildModifyColumnRequest(tableName, columnFamily, nonceGroup, nonce); + return master.modifyColumn(getRpcController(), req); + } + }); return new ModifyColumnFamilyFuture(this, tableName, response); } @@ -1108,6 +1146,39 @@ public String getOperationType() { } } + @Override + public Future modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family, + String dstSFT) throws IOException { + ModifyColumnStoreFileTrackerResponse response = + executeCallable(new MasterCallable(getConnection(), + getRpcControllerFactory()) { + long nonceGroup = ng.getNonceGroup(); + long nonce = ng.newNonce(); + + @Override + protected ModifyColumnStoreFileTrackerResponse rpcCall() throws Exception { + setPriority(tableName); + ModifyColumnStoreFileTrackerRequest req = RequestConverter + .buildModifyColumnStoreFileTrackerRequest(tableName, family, dstSFT, nonceGroup, nonce); + return master.modifyColumnStoreFileTracker(getRpcController(), req); + } + }); + return new ModifyColumnFamilyStoreFileTrackerFuture(this, tableName, response); + } + + private static class ModifyColumnFamilyStoreFileTrackerFuture extends ModifyTableFuture { + public ModifyColumnFamilyStoreFileTrackerFuture(HBaseAdmin admin, TableName tableName, + final ModifyColumnStoreFileTrackerResponse response) { + super(admin, tableName, + (response != null && response.hasProcId()) ? response.getProcId() : null); + } + + @Override + public String getOperationType() { + return "MODIFY_COLUMN_FAMILY_STORE_FILE_TRACKER"; + } + } + @Deprecated @Override public void closeRegion(final String regionName, final String unused) throws IOException { 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 64b82cd3c223..3ae948bb8d50 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 @@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException; + import com.google.protobuf.Message; import com.google.protobuf.RpcChannel; import edu.umd.cs.findbugs.annotations.Nullable; @@ -46,7 +47,6 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.AsyncMetaTableAccessor; import org.apache.hadoop.hbase.CacheEvictionStats; @@ -89,6 +89,23 @@ import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; import org.apache.hadoop.hbase.security.access.UserPermission; +import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; +import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; +import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.ForeignExceptionUtil; +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.protobuf.RpcCallback; +import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; +import org.apache.hbase.thirdparty.io.netty.util.Timeout; +import org.apache.hbase.thirdparty.io.netty.util.TimerTask; +import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos; @@ -211,10 +228,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; @@ -281,21 +302,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; -import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; -import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ForeignExceptionUtil; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; -import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; -import org.apache.hbase.thirdparty.io.netty.util.Timeout; -import org.apache.hbase.thirdparty.io.netty.util.TimerTask; -import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * The implementation of AsyncAdmin. @@ -626,6 +632,18 @@ public CompletableFuture modifyTable(TableDescriptor desc) { (resp) -> resp.getProcId(), new ModifyTableProcedureBiConsumer(this, desc.getTableName())); } + @Override + public CompletableFuture modifyTableStoreFileTracker(TableName tableName, String dstSFT) { + return this + . procedureCall( + tableName, + RequestConverter.buildModifyTableStoreFileTrackerRequest(tableName, dstSFT, + ng.getNonceGroup(), ng.newNonce()), + (s, c, req, done) -> s.modifyTableStoreFileTracker(c, req, done), + (resp) -> resp.getProcId(), + new ModifyTableStoreFileTrackerProcedureBiConsumer(this, tableName)); + } + @Override public CompletableFuture deleteTable(TableName tableName) { return this. procedureCall(tableName, @@ -806,6 +824,19 @@ public CompletableFuture modifyColumnFamily(TableName tableName, (resp) -> resp.getProcId(), new ModifyColumnFamilyProcedureBiConsumer(tableName)); } + @Override + public CompletableFuture modifyColumnFamilyStoreFileTracker(TableName tableName, + byte[] family, String dstSFT) { + return this + . procedureCall( + tableName, + RequestConverter.buildModifyColumnStoreFileTrackerRequest(tableName, family, dstSFT, + ng.getNonceGroup(), ng.newNonce()), + (s, c, req, done) -> s.modifyColumnStoreFileTracker(c, req, done), + (resp) -> resp.getProcId(), + new ModifyColumnFamilyStoreFileTrackerProcedureBiConsumer(tableName)); + } + @Override public CompletableFuture createNamespace(NamespaceDescriptor descriptor) { return this. procedureCall( @@ -2556,7 +2587,20 @@ private static class ModifyTableProcedureBiConsumer extends TableProcedureBiCons @Override String getOperationType() { - return "ENABLE"; + return "MODIFY"; + } + } + + private static class ModifyTableStoreFileTrackerProcedureBiConsumer + extends TableProcedureBiConsumer { + + ModifyTableStoreFileTrackerProcedureBiConsumer(AsyncAdmin admin, TableName tableName) { + super(tableName); + } + + @Override + String getOperationType() { + return "MODIFY_TABLE_STORE_FILE_TRACKER"; } } @@ -2650,6 +2694,19 @@ String getOperationType() { } } + private static class ModifyColumnFamilyStoreFileTrackerProcedureBiConsumer + extends TableProcedureBiConsumer { + + ModifyColumnFamilyStoreFileTrackerProcedureBiConsumer(TableName tableName) { + super(tableName); + } + + @Override + String getOperationType() { + return "MODIFY_COLUMN_FAMILY_STORE_FILE_TRACKER"; + } + } + private static class CreateNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer { CreateNamespaceProcedureBiConsumer(String namespaceName) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java index b2055041f7cf..97127e0e3ebd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java @@ -126,10 +126,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; @@ -729,4 +733,16 @@ public HasUserPermissionsResponse hasUserPermissions(RpcController controller, HasUserPermissionsRequest request) throws ServiceException { return stub.hasUserPermissions(controller, request); } + + @Override + public ModifyTableStoreFileTrackerResponse modifyTableStoreFileTracker(RpcController controller, + ModifyTableStoreFileTrackerRequest request) throws ServiceException { + return stub.modifyTableStoreFileTracker(controller, request); + } + + @Override + public ModifyColumnStoreFileTrackerResponse modifyColumnStoreFileTracker(RpcController controller, + ModifyColumnStoreFileTrackerRequest request) throws ServiceException { + return stub.modifyColumnStoreFileTracker(controller, request); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java index 5f2181e3ccd1..d00627a0e572 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java @@ -26,7 +26,6 @@ import java.util.Set; import java.util.regex.Pattern; import java.util.stream.Collectors; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.ClusterMetrics.Option; @@ -68,6 +67,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils; @@ -132,8 +132,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest; @@ -1246,6 +1248,19 @@ public static ModifyColumnRequest buildModifyColumnRequest( return builder.build(); } + public static ModifyColumnStoreFileTrackerRequest + buildModifyColumnStoreFileTrackerRequest(final TableName tableName, final byte[] family, + final String dstSFT, final long nonceGroup, final long nonce) { + ModifyColumnStoreFileTrackerRequest.Builder builder = + ModifyColumnStoreFileTrackerRequest.newBuilder(); + builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); + builder.setFamily(ByteString.copyFrom(family)); + builder.setDstSft(dstSFT); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); + return builder.build(); + } + /** * Create a protocol buffer MoveRegionRequest * @param encodedRegionName @@ -1444,6 +1459,17 @@ public static ModifyTableRequest buildModifyTableRequest( return builder.build(); } + public static ModifyTableStoreFileTrackerRequest buildModifyTableStoreFileTrackerRequest( + final TableName tableName, final String dstSFT, final long nonceGroup, final long nonce) { + ModifyTableStoreFileTrackerRequest.Builder builder = + ModifyTableStoreFileTrackerRequest.newBuilder(); + builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); + builder.setDstSft(dstSFT); + builder.setNonceGroup(nonceGroup); + builder.setNonce(nonce); + return builder.build(); + } + /** * Creates a protocol buffer GetSchemaAlterStatusRequest * diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto index 8eccb7a0b084..427645545903 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto @@ -731,6 +731,29 @@ message BalancerRejectionsResponse { repeated BalancerRejection balancer_rejection = 1; } +message ModifyTableStoreFileTrackerRequest { + required TableName table_Name = 1; + required string dst_sft = 2; + optional uint64 nonce_group = 3 [default = 0]; + optional uint64 nonce = 4 [default = 0]; +} + +message ModifyTableStoreFileTrackerResponse { + optional uint64 proc_id = 1; +} + +message ModifyColumnStoreFileTrackerRequest { + required TableName table_Name = 1; + required bytes family = 2; + required string dst_sft = 3; + optional uint64 nonce_group = 4 [default = 0]; + optional uint64 nonce = 5 [default = 0]; +} + +message ModifyColumnStoreFileTrackerResponse { + optional uint64 proc_id = 1; +} + service MasterService { /** Used by the client to get the number of regions that have received the updated schema */ rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest) @@ -1121,6 +1144,12 @@ service MasterService { rpc GetLogEntries(LogRequest) returns(LogEntry); + + rpc ModifyTableStoreFileTracker(ModifyTableStoreFileTrackerRequest) + returns(ModifyTableStoreFileTrackerResponse); + + rpc ModifyColumnStoreFileTracker(ModifyColumnStoreFileTrackerRequest) + returns(ModifyColumnStoreFileTrackerResponse); } // HBCK Service definitions. diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index fdcb1818d367..593fe2316336 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -596,3 +596,18 @@ message ModifyTableDescriptorStateData { required TableName table_name = 1; optional TableSchema modified_table_schema = 2; } + +enum ModifyStoreFileTrackerState { + MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION = 1; + MODIFY_STORE_FILE_TRACKER_START_MIGRATION = 2; + MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION = 3; +} + +message ModifyStoreFileTrackerStateData { + required TableName table_name = 1; + required string dst_sft = 2; +} + +message ModifyColumnFamilyStoreFileTrackerStateData { + required bytes family = 1; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java index ce70647e9254..bf667eb8ea20 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java @@ -291,6 +291,58 @@ default void preModifyTableAction( final TableName tableName, final TableDescriptor newDescriptor) throws IOException {} + /** + * Called prior to modifying a table's store file tracker. Called as part of modify + * table store file tracker RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param dstSFT the store file tracker + * @return the store file tracker + */ + default String preModifyTableStoreFileTracker( + final ObserverContext ctx, final TableName tableName, + String dstSFT) throws IOException { + return dstSFT; + } + + /** + * Called after modifying a table's store file tracker. Called as part of modify + * table store file tracker RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param dstSFT the store file tracker + */ + default void postModifyTableStoreFileTracker( + final ObserverContext ctx, final TableName tableName, + String dstSFT) throws IOException {} + + /** + * Called prior to modifying a family's store file tracker. Called as part of modify family store + * file tracker RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param family the column family + * @param dstSFT the store file tracker + * @return the store file tracker + */ + default String preModifyColumnFamilyStoreFileTracker( + final ObserverContext ctx, final TableName tableName, + final byte[] family, String dstSFT) throws IOException { + return dstSFT; + } + + /** + * Called after modifying a family store file tracker. Called as part of modify family store + * file tracker RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param family the column family + * @param dstSFT the store file tracker + */ + default void postModifyColumnFamilyStoreFileTracker( + final ObserverContext ctx, final TableName tableName, + final byte[] family, String dstSFT) throws IOException {} + /** * Called prior to modifying a table's properties. Called as part of modify * table procedure and it is async to the modify table RPC call. 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 8fe4324cb58e..5d82a7c183a0 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 @@ -195,6 +195,8 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.regionserver.RSRpcServices; +import org.apache.hadoop.hbase.regionserver.storefiletracker.ModifyColumnFamilyStoreFileTrackerProcedure; +import org.apache.hadoop.hbase.regionserver.storefiletracker.ModifyTableStoreFileTrackerProcedure; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationLoadSource; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; @@ -2449,6 +2451,33 @@ public TableDescriptor get() throws IOException { }, nonceGroup, nonce, true); } + + @Override + public long modifyColumnStoreFileTracker(TableName tableName, byte[] family, String dstSFT, + long nonceGroup, long nonce) throws IOException { + checkInitialized(); + return MasterProcedureUtil + .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + + @Override + protected void run() throws IOException { + String sft = getMaster().getMasterCoprocessorHost() + .preModifyColumnFamilyStoreFileTracker(tableName, family, dstSFT); + LOG.info("{} modify column {} store file tracker of table {} to {}", + getClientIdAuditPrefix(), Bytes.toStringBinary(family), tableName, sft); + submitProcedure(new ModifyColumnFamilyStoreFileTrackerProcedure( + procedureExecutor.getEnvironment(), tableName, family, sft)); + getMaster().getMasterCoprocessorHost().postModifyColumnFamilyStoreFileTracker(tableName, + family, dstSFT); + } + + @Override + protected String getDescription() { + return "ModifyColumnFamilyStoreFileTrackerProcedure"; + } + }); + } + @Override public long deleteColumn(final TableName tableName, final byte[] columnName, final long nonceGroup, final long nonce) throws IOException { @@ -2614,6 +2643,31 @@ public TableDescriptor get() throws IOException { } + @Override + public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup, + long nonce) throws IOException { + checkInitialized(); + return MasterProcedureUtil + .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + + @Override + protected void run() throws IOException { + String sft = getMaster().getMasterCoprocessorHost() + .preModifyTableStoreFileTracker(tableName, dstSFT); + LOG.info("{} modify table store file tracker of table {} to {}", getClientIdAuditPrefix(), + tableName, sft); + submitProcedure(new ModifyTableStoreFileTrackerProcedure( + procedureExecutor.getEnvironment(), tableName, sft)); + getMaster().getMasterCoprocessorHost().postModifyTableStoreFileTracker(tableName, sft); + } + + @Override + protected String getDescription() { + return "ModifyTableStoreFileTrackerProcedure"; + } + }); + } + public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long nonceGroup, final long nonce, final boolean restoreAcl, final String customSFT) throws IOException { checkInitialized(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java index a1f2dce361c7..84e2eb19ba05 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java @@ -493,6 +493,55 @@ public void call(MasterObserver observer) throws IOException { }); } + public String preModifyTableStoreFileTracker(final TableName tableName, final String dstSFT) + throws IOException { + if (coprocEnvironments.isEmpty()) { + return dstSFT; + } + return execOperationWithResult( + new ObserverOperationWithResult(masterObserverGetter, dstSFT) { + @Override + protected String call(MasterObserver observer) throws IOException { + return observer.preModifyTableStoreFileTracker(this, tableName, getResult()); + } + }); + } + + public void postModifyTableStoreFileTracker(final TableName tableName, final String dstSFT) + throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() { + @Override + public void call(MasterObserver observer) throws IOException { + observer.postModifyTableStoreFileTracker(this, tableName, dstSFT); + } + }); + } + + public String preModifyColumnFamilyStoreFileTracker(final TableName tableName, + final byte[] family, final String dstSFT) throws IOException { + if (coprocEnvironments.isEmpty()) { + return dstSFT; + } + return execOperationWithResult( + new ObserverOperationWithResult(masterObserverGetter, dstSFT) { + @Override + protected String call(MasterObserver observer) throws IOException { + return observer.preModifyColumnFamilyStoreFileTracker(this, tableName, family, + getResult()); + } + }); + } + + public void postModifyColumnFamilyStoreFileTracker(final TableName tableName, final byte[] family, + final String dstSFT) throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() { + @Override + public void call(MasterObserver observer) throws IOException { + observer.postModifyColumnFamilyStoreFileTracker(this, tableName, family, dstSFT); + } + }); + } + public void preModifyTableAction(final TableName tableName, final TableDescriptor currentDescriptor, final TableDescriptor newDescriptor, final User user) throws IOException { 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 695c960637ac..abfa44f4bb8d 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 @@ -260,10 +260,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; @@ -1405,6 +1409,20 @@ public ModifyColumnResponse modifyColumn(RpcController controller, } } + @Override + public ModifyColumnStoreFileTrackerResponse modifyColumnStoreFileTracker( + RpcController controller, ModifyColumnStoreFileTrackerRequest req) + throws ServiceException { + try { + long procId = + master.modifyColumnStoreFileTracker(ProtobufUtil.toTableName(req.getTableName()), + req.getFamily().toByteArray(), req.getDstSft(), req.getNonceGroup(), req.getNonce()); + return ModifyColumnStoreFileTrackerResponse.newBuilder().setProcId(procId).build(); + } catch (IOException ioe) { + throw new ServiceException(ioe); + } + } + @Override public ModifyNamespaceResponse modifyNamespace(RpcController controller, ModifyNamespaceRequest request) throws ServiceException { @@ -1434,6 +1452,18 @@ public ModifyTableResponse modifyTable(RpcController controller, } } + @Override + public ModifyTableStoreFileTrackerResponse modifyTableStoreFileTracker(RpcController controller, + ModifyTableStoreFileTrackerRequest req) throws ServiceException { + try { + long procId = master.modifyTableStoreFileTracker(ProtobufUtil.toTableName(req.getTableName()), + req.getDstSft(), req.getNonceGroup(), req.getNonce()); + return ModifyTableStoreFileTrackerResponse.newBuilder().setProcId(procId).build(); + } catch (IOException ioe) { + throw new ServiceException(ioe); + } + } + @Override public MoveRegionResponse moveRegion(RpcController controller, MoveRegionRequest req) throws ServiceException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 3543ed99e254..9370a5707a96 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -216,6 +216,15 @@ long modifyTable( final long nonce) throws IOException; + /** + * Modify the store file tracker of an existing table + */ + long modifyTableStoreFileTracker( + final TableName tableName, + final String dstSFT, + final long nonceGroup, + final long nonce) throws IOException; + /** * Enable an existing table * @param tableName The table name @@ -271,6 +280,16 @@ long modifyColumn( final long nonce) throws IOException; + /** + * Modify the store file tracker of an existing column in an existing table + */ + long modifyColumnStoreFileTracker( + final TableName tableName, + final byte[] family, + final String dstSFT, + final long nonceGroup, + final long nonce) throws IOException; + /** * Delete a column from an existing table * @param tableName The table name 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 441fddbfe7ce..a1e39858193b 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 @@ -36,6 +36,7 @@ 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.regionserver.storefiletracker.StoreFileTrackerValidationUtils; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.FSTableDescriptors; @@ -256,7 +257,8 @@ private boolean prepareCreate(final MasterProcedureEnv env) throws IOException { } // check for store file tracker configurations - StoreFileTrackerFactory.checkForCreateTable(env.getMasterConfiguration(), tableDescriptor); + StoreFileTrackerValidationUtils.checkForCreateTable(env.getMasterConfiguration(), + tableDescriptor); return true; } 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 e58fc92574c9..b8e6b496995b 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 @@ -37,7 +37,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.regionserver.storefiletracker.StoreFileTrackerValidationUtils; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; @@ -320,7 +320,7 @@ private void prepareModify(final MasterProcedureEnv env) throws IOException { modifiedTableDescriptor); // check for store file tracker configurations - StoreFileTrackerFactory.checkForModifyTable(env.getMasterConfiguration(), + StoreFileTrackerValidationUtils.checkForModifyTable(env.getMasterConfiguration(), unmodifiedTableDescriptor, 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 9196a97fafa3..263caf5820d9 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 @@ -66,7 +66,7 @@ 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.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerValidationUtils; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessChecker; @@ -884,8 +884,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 - StoreFileTrackerFactory.validatePreRestoreSnapshot(master.getTableDescriptors().get(tableName), - snapshotTableDesc, master.getConfiguration()); + StoreFileTrackerValidationUtils.validatePreRestoreSnapshot( + 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/ModifyColumnFamilyStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyColumnFamilyStoreFileTrackerProcedure.java new file mode 100644 index 000000000000..1ecfee26e252 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyColumnFamilyStoreFileTrackerProcedure.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.regionserver.storefiletracker; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseIOException; +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.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; +import org.apache.hadoop.hbase.regionserver.StoreUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStoreFileTrackerStateData; + +@InterfaceAudience.Private +public class ModifyColumnFamilyStoreFileTrackerProcedure extends ModifyStoreFileTrackerProcedure { + + private byte[] family; + + public ModifyColumnFamilyStoreFileTrackerProcedure() { + } + + public ModifyColumnFamilyStoreFileTrackerProcedure(MasterProcedureEnv env, TableName tableName, + byte[] family, String dstSFT) throws HBaseIOException { + super(env, tableName, dstSFT); + this.family = family; + } + + @Override + protected void preCheck(TableDescriptor current) throws IOException { + if (!current.hasColumnFamily(family)) { + throw new NoSuchColumnFamilyException( + Bytes.toStringBinary(family) + " does not exist for table " + current.getTableName()); + } + } + + @Override + protected Configuration createConf(Configuration conf, TableDescriptor current) { + ColumnFamilyDescriptor cfd = current.getColumnFamily(family); + return StoreUtils.createStoreConfiguration(conf, current, cfd); + } + + @Override + protected TableDescriptor createRestoreTableDescriptor(TableDescriptor current, + String restoreSFT) { + ColumnFamilyDescriptor cfd = + ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family)) + .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, restoreSFT).build(); + return TableDescriptorBuilder.newBuilder(current).modifyColumnFamily(cfd).build(); + } + + @Override + protected TableDescriptor createMigrationTableDescriptor(Configuration conf, + TableDescriptor current) { + ColumnFamilyDescriptorBuilder builder = + ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family)); + migrate(conf, builder::setConfiguration); + return TableDescriptorBuilder.newBuilder(current).modifyColumnFamily(builder.build()).build(); + } + + @Override + protected TableDescriptor createFinishTableDescriptor(TableDescriptor current) { + ColumnFamilyDescriptorBuilder builder = + ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family)); + finish(builder::setConfiguration, builder::removeConfiguration); + return TableDescriptorBuilder.newBuilder(current).modifyColumnFamily(builder.build()).build(); + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + serializer.serialize(ModifyColumnFamilyStoreFileTrackerStateData.newBuilder() + .setFamily(ByteString.copyFrom(family)).build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + ModifyColumnFamilyStoreFileTrackerStateData data = + serializer.deserialize(ModifyColumnFamilyStoreFileTrackerStateData.class); + this.family = data.getFamily().toByteArray(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyStoreFileTrackerProcedure.java new file mode 100644 index 000000000000..a7d8e703acc3 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyStoreFileTrackerProcedure.java @@ -0,0 +1,271 @@ +/** + * 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.function.BiConsumer; +import java.util.function.Consumer; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNotEnabledException; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyStoreFileTrackerState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyStoreFileTrackerStateData; + +/** + * This procedure is used to change the store file tracker implementation. + *

+ * Typically we need to schedule two {@link ModifyTableProcedure} (or three if the table is already + * in {@code MIGRATION} but the {@code dstSFT} is not what we expected) to do this, so we introduce + * this procedure to simplify the work of our users. + */ +@InterfaceAudience.Private +public abstract class ModifyStoreFileTrackerProcedure + extends AbstractStateMachineTableProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(ModifyStoreFileTrackerProcedure.class); + + private TableName tableName; + + private String dstSFT; + + protected ModifyStoreFileTrackerProcedure() { + } + + protected ModifyStoreFileTrackerProcedure(MasterProcedureEnv env, TableName tableName, + String dstSFT) throws HBaseIOException { + super(env); + checkDstSFT(dstSFT); + this.tableName = tableName; + this.dstSFT = dstSFT; + preflightChecks(env, true); + } + + private void checkDstSFT(String dstSFT) throws DoNotRetryIOException { + if (MigrationStoreFileTracker.class + .isAssignableFrom(StoreFileTrackerFactory.getTrackerClass(dstSFT))) { + throw new DoNotRetryIOException("Do not need to transfer to " + dstSFT); + } + } + + @Override + public TableName getTableName() { + return tableName; + } + + @Override + public TableOperationType getTableOperationType() { + return TableOperationType.EDIT; + } + + private enum StoreFileTrackerState { + NEED_FINISH_PREVIOUS_MIGRATION_FIRST, NEED_START_MIGRATION, NEED_FINISH_MIGRATION, + ALREADY_FINISHED + } + + private StoreFileTrackerState checkState(Configuration conf, String dstSFT) { + // there are 4 possible conditions: + // 1. The table or family has already made use of the dstSFT. In this way we just finish the + // procedure. + // 2. The table or family is not using the dstSFT but also not using migration SFT, + // then we just go to the MODIFY_STORE_FILE_TRACKER_MIGRATION state. + // 3. The table or family has already been using migration SFT and the dst SFT is what we + // expect, just go to MODIFY_STORE_FILE_TRACKER_FINISH. + // 4. The table or family is using migration SFT and the dst SFT is not what we + // expect, then need to schedule a MTP to change it to the dst SFT of the current migration + // SFT first, and then go to MODIFY_STORE_FILE_TRACKER_MIGRATION. + Class clazz = StoreFileTrackerFactory.getTrackerClass(conf); + Class dstSFTClass = StoreFileTrackerFactory.getTrackerClass(dstSFT); + if (clazz.equals(dstSFTClass)) { + return StoreFileTrackerState.ALREADY_FINISHED; + } + if (!MigrationStoreFileTracker.class.isAssignableFrom(clazz)) { + return StoreFileTrackerState.NEED_START_MIGRATION; + } + Class currentDstSFT = StoreFileTrackerFactory + .getStoreFileTrackerClassForMigration(conf, MigrationStoreFileTracker.DST_IMPL); + if (currentDstSFT.equals(dstSFTClass)) { + return StoreFileTrackerState.NEED_FINISH_MIGRATION; + } else { + return StoreFileTrackerState.NEED_FINISH_PREVIOUS_MIGRATION_FIRST; + } + } + + private final String getRestoreSFT(Configuration conf) { + Class currentDstSFT = StoreFileTrackerFactory + .getStoreFileTrackerClassForMigration(conf, MigrationStoreFileTracker.DST_IMPL); + return StoreFileTrackerFactory.getStoreFileTrackerName(currentDstSFT); + } + + protected abstract void preCheck(TableDescriptor current) throws IOException; + + protected abstract Configuration createConf(Configuration conf, TableDescriptor current); + + protected abstract TableDescriptor createRestoreTableDescriptor(TableDescriptor current, + String restoreSFT); + + private Flow preCheckAndTryRestoreSFT(MasterProcedureEnv env) throws IOException { + // Checks whether the table exists + if (!env.getMasterServices().getTableDescriptors().exists(getTableName())) { + throw new TableNotFoundException(getTableName()); + } + if (!isTableEnabled(env)) { + throw new TableNotEnabledException(tableName); + } + TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName); + preCheck(current); + Configuration conf = createConf(env.getMasterConfiguration(), current); + StoreFileTrackerState state = checkState(conf, dstSFT); + switch (state) { + case NEED_FINISH_PREVIOUS_MIGRATION_FIRST: + TableDescriptor td = createRestoreTableDescriptor(current, getRestoreSFT(conf)); + addChildProcedure(new ModifyTableProcedure(env, td)); + setNextState( + ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION); + return Flow.HAS_MORE_STATE; + case NEED_START_MIGRATION: + setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_START_MIGRATION); + return Flow.HAS_MORE_STATE; + case NEED_FINISH_MIGRATION: + setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION); + return Flow.HAS_MORE_STATE; + case ALREADY_FINISHED: + return Flow.NO_MORE_STATE; + default: + throw new UnsupportedOperationException("unhandled state=" + state); + } + } + + protected abstract TableDescriptor createMigrationTableDescriptor(Configuration conf, + TableDescriptor current); + + protected final void migrate(Configuration conf, BiConsumer setValue) { + setValue.accept(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()); + setValue.accept(MigrationStoreFileTracker.SRC_IMPL, + StoreFileTrackerFactory.getStoreFileTrackerName(conf)); + setValue.accept(MigrationStoreFileTracker.DST_IMPL, dstSFT); + } + + protected abstract TableDescriptor createFinishTableDescriptor(TableDescriptor current); + + protected final void finish(BiConsumer setValue, Consumer removeValue) { + setValue.accept(StoreFileTrackerFactory.TRACKER_IMPL, dstSFT); + removeValue.accept(MigrationStoreFileTracker.SRC_IMPL); + removeValue.accept(MigrationStoreFileTracker.DST_IMPL); + } + + private void migrate(MasterProcedureEnv env) throws IOException { + TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName); + TableDescriptor td = createMigrationTableDescriptor(env.getMasterConfiguration(), current); + addChildProcedure(new ModifyTableProcedure(env, td)); + setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION); + } + + private void finish(MasterProcedureEnv env) throws IOException { + TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName); + TableDescriptor td = createFinishTableDescriptor(current); + addChildProcedure(new ModifyTableProcedure(env, td)); + } + + @Override + protected Flow executeFromState(MasterProcedureEnv env, ModifyStoreFileTrackerState state) + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { + try { + switch (state) { + case MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION: + return preCheckAndTryRestoreSFT(env); + case MODIFY_STORE_FILE_TRACKER_START_MIGRATION: + migrate(env); + return Flow.HAS_MORE_STATE; + case MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION: + finish(env); + return Flow.NO_MORE_STATE; + default: + throw new UnsupportedOperationException("unhandled state=" + state); + } + } catch (IOException e) { + if (isRollbackSupported(state)) { + setFailure("master-modify-SFT", e); + } else { + LOG.warn("Retriable error trying to modify SFT for table={} (in state={})", getTableName(), + state, e); + } + } + return Flow.HAS_MORE_STATE; + } + + @Override + protected void rollbackState(MasterProcedureEnv env, ModifyStoreFileTrackerState state) + throws IOException, InterruptedException { + if (isRollbackSupported(state)) { + return; + } + throw new UnsupportedOperationException("unhandled state=" + state); + } + + @Override + protected ModifyStoreFileTrackerState getState(int stateId) { + return ModifyStoreFileTrackerState.forNumber(stateId); + } + + @Override + protected int getStateId(ModifyStoreFileTrackerState state) { + return state.getNumber(); + } + + @Override + protected ModifyStoreFileTrackerState getInitialState() { + return ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION; + } + + @Override + protected boolean isRollbackSupported(ModifyStoreFileTrackerState state) { + return state == ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + serializer.serialize(ModifyStoreFileTrackerStateData.newBuilder() + .setTableName(ProtobufUtil.toProtoTableName(tableName)).setDstSft(dstSFT).build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + ModifyStoreFileTrackerStateData data = + serializer.deserialize(ModifyStoreFileTrackerStateData.class); + this.tableName = ProtobufUtil.toTableName(data.getTableName()); + this.dstSFT = data.getDstSft(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyTableStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyTableStoreFileTrackerProcedure.java new file mode 100644 index 000000000000..096f38fa36eb --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyTableStoreFileTrackerProcedure.java @@ -0,0 +1,71 @@ +/** + * 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.CompoundConfiguration; +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class ModifyTableStoreFileTrackerProcedure extends ModifyStoreFileTrackerProcedure { + + public ModifyTableStoreFileTrackerProcedure() { + } + + public ModifyTableStoreFileTrackerProcedure(MasterProcedureEnv env, TableName tableName, + String dstSFT) throws HBaseIOException { + super(env, tableName, dstSFT); + } + + @Override + protected void preCheck(TableDescriptor current) { + } + + @Override + protected Configuration createConf(Configuration conf, TableDescriptor current) { + return new CompoundConfiguration().add(conf).addBytesMap(current.getValues()); + } + + @Override + protected TableDescriptor createRestoreTableDescriptor(TableDescriptor current, + String restoreSFT) { + return TableDescriptorBuilder.newBuilder(current) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, restoreSFT).build(); + } + + @Override + protected TableDescriptor createMigrationTableDescriptor(Configuration conf, + TableDescriptor current) { + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(current); + migrate(conf, builder::setValue); + return builder.build(); + } + + @Override + protected TableDescriptor createFinishTableDescriptor(TableDescriptor current) { + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(current); + finish(builder::setValue, builder::removeValue); + return builder.build(); + } + +} 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 61a71c20f8b8..7f545e4058d7 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,21 +15,17 @@ */ 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.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.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -103,6 +99,20 @@ public static Class getTrackerClass(Configuration co } } + public static Class getTrackerClass(String trackerNameOrClass) { + try { + Trackers tracker = Trackers.valueOf(trackerNameOrClass.toUpperCase()); + return tracker.clazz; + } catch (IllegalArgumentException e) { + // Fall back to them specifying a class name + try { + return Class.forName(trackerNameOrClass).asSubclass(StoreFileTracker.class); + } catch (ClassNotFoundException e1) { + throw new RuntimeException(e1); + } + } + } + public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) { Class tracker = getTrackerClass(conf); @@ -174,178 +184,4 @@ public static TableDescriptor updateWithTrackerConfigs(Configuration conf, } return descriptor; } - - // 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()); - } - } - } - } - } - - /** - * 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 validatePreRestoreSnapshot(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/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java new file mode 100644 index 000000000000..e6f6e854c890 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java @@ -0,0 +1,212 @@ +/** + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.regionserver.StoreUtils; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.Trackers; +import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public final class StoreFileTrackerValidationUtils { + + private StoreFileTrackerValidationUtils() { + } + + // should not use MigrationStoreFileTracker for new family + private static void checkForNewFamily(Configuration conf, TableDescriptor table, + ColumnFamilyDescriptor family) throws IOException { + Configuration mergedConf = StoreUtils.createStoreConfiguration(conf, table, family); + Class tracker = StoreFileTrackerFactory.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 = StoreUtils.createStoreConfiguration(conf, oldTable, oldFamily); + Configuration newConf = StoreUtils.createStoreConfiguration(conf, newTable, newFamily); + + Class oldTracker = + StoreFileTrackerFactory.getTrackerClass(oldConf); + Class newTracker = + StoreFileTrackerFactory.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 " + + StoreFileTrackerFactory.getStoreFileTrackerName(oldSrcTracker) + " to " + + StoreFileTrackerFactory.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 " + + StoreFileTrackerFactory.getStoreFileTrackerName(oldDstTracker) + " to " + + StoreFileTrackerFactory.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 " + + StoreFileTrackerFactory.getStoreFileTrackerName(oldDstTracker) + " but got " + + StoreFileTrackerFactory.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 " + + StoreFileTrackerFactory.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 " + + StoreFileTrackerFactory.getStoreFileTrackerName(oldTracker) + " first but got " + + StoreFileTrackerFactory.getStoreFileTrackerName(newSrcTracker) + + " when migrating from " + + StoreFileTrackerFactory.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 " + + StoreFileTrackerFactory.getStoreFileTrackerName(newSrcTracker) + " for family " + + newFamily.getNameAsString() + " of table " + newTable.getTableName()); + } + } + } + } + } + + /** + * 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 validatePreRestoreSnapshot(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/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index e8092992bac1..82eb317ec48a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -37,7 +37,6 @@ import java.util.TreeMap; import java.util.TreeSet; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; @@ -125,7 +124,6 @@ import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.access.Permission.Action; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; import org.apache.hadoop.hbase.util.ByteRange; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -142,6 +140,8 @@ import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; +import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; + /** * Provides basic authorization checks for data access and administrative * operations. @@ -921,6 +921,22 @@ public Void run() throws Exception { }); } + public String preModifyTableStoreFileTracker(ObserverContext c, + TableName tableName, String dstSFT) throws IOException { + requirePermission(c, "modifyTableStoreFileTracker", tableName, null, null, Action.ADMIN, + Action.CREATE); + return dstSFT; + } + + @Override + public String preModifyColumnFamilyStoreFileTracker( + ObserverContext c, TableName tableName, byte[] family, + String dstSFT) throws IOException { + requirePermission(c, "modifyColumnFamilyStoreFileTracker", tableName, family, null, + Action.ADMIN, Action.CREATE); + return dstSFT; + } + @Override public void preEnableTable(ObserverContext c, TableName tableName) throws IOException { 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 c2de0fbd3555..9657fda882d3 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 @@ -18,14 +18,18 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; import java.util.List; import java.util.regex.Pattern; +import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -35,6 +39,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.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -393,4 +398,156 @@ public void testDeleteEditUnknownColumnFamilyAndOrTable() throws IOException { ADMIN.deleteTable(tableName); } } + + private static final String SRC_IMPL = "hbase.store.file-tracker.migration.src.impl"; + + private static final String DST_IMPL = "hbase.store.file-tracker.migration.dst.impl"; + + private void verifyModifyTableResult(TableName tableName, byte[] family, byte[] qual, byte[] row, + byte[] value, String sft) throws IOException { + TableDescriptor td = ADMIN.getDescriptor(tableName); + assertEquals(sft, td.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + // no migration related configs + assertNull(td.getValue(SRC_IMPL)); + assertNull(td.getValue(DST_IMPL)); + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual)); + } + } + + @Test + public void testModifyTableStoreFileTracker() throws IOException { + TableName tableName = TableName.valueOf(name.getMethodName()); + byte[] family = Bytes.toBytes("info"); + byte[] qual = Bytes.toBytes("q"); + byte[] row = Bytes.toBytes(0); + byte[] value = Bytes.toBytes(1); + try (Table table = TEST_UTIL.createTable(tableName, family)) { + table.put(new Put(row).addColumn(family, qual, value)); + } + // change to FILE + ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name()); + verifyModifyTableResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to FILE again, should have no effect + ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name()); + verifyModifyTableResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to MIGRATION, and then to FILE + ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName)) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(SRC_IMPL, + StoreFileTrackerFactory.Trackers.FILE.name()) + .setValue(DST_IMPL, + StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .build()); + ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name()); + verifyModifyTableResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to MIGRATION, and then to DEFAULT + ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName)) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(SRC_IMPL, + StoreFileTrackerFactory.Trackers.FILE.name()) + .setValue(DST_IMPL, + StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .build()); + ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.DEFAULT.name()); + verifyModifyTableResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.DEFAULT.name()); + } + + private void verifyModifyColumnFamilyResult(TableName tableName, byte[] family, byte[] qual, + byte[] row, byte[] value, String sft) throws IOException { + TableDescriptor td = ADMIN.getDescriptor(tableName); + ColumnFamilyDescriptor cfd = td.getColumnFamily(family); + assertEquals(sft, cfd.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL)); + // no migration related configs + assertNull(cfd.getConfigurationValue(SRC_IMPL)); + assertNull(cfd.getConfigurationValue(DST_IMPL)); + assertNull(cfd.getValue(SRC_IMPL)); + assertNull(cfd.getValue(DST_IMPL)); + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual)); + } + } + + @Test + public void testModifyColumnFamilyStoreFileTracker() throws IOException { + TableName tableName = TableName.valueOf(name.getMethodName()); + byte[] family = Bytes.toBytes("info"); + byte[] qual = Bytes.toBytes("q"); + byte[] row = Bytes.toBytes(0); + byte[] value = Bytes.toBytes(1); + try (Table table = TEST_UTIL.createTable(tableName, family)) { + table.put(new Put(row).addColumn(family, qual, value)); + } + // change to FILE + ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family, + StoreFileTrackerFactory.Trackers.FILE.name()); + verifyModifyColumnFamilyResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to FILE again, should have no effect + ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family, + StoreFileTrackerFactory.Trackers.FILE.name()); + verifyModifyColumnFamilyResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to MIGRATION, and then to FILE + TableDescriptor current = ADMIN.getDescriptor(tableName); + ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(current) + .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family)) + .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build()) + .build()); + ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family, + StoreFileTrackerFactory.Trackers.FILE.name()); + verifyModifyColumnFamilyResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to MIGRATION, and then to DEFAULT + current = ADMIN.getDescriptor(tableName); + ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(current) + .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family)) + .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build()) + .build()); + ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family, + StoreFileTrackerFactory.Trackers.DEFAULT.name()); + verifyModifyColumnFamilyResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.DEFAULT.name()); + } + + @Test + public void testModifyStoreFileTrackerError() throws IOException { + TableName tableName = TableName.valueOf(name.getMethodName()); + byte[] family = Bytes.toBytes("info"); + TEST_UTIL.createTable(tableName, family).close(); + + // table not exists + assertThrows(TableNotFoundException.class, + () -> ADMIN.modifyTableStoreFileTracker(TableName.valueOf("whatever"), + StoreFileTrackerFactory.Trackers.FILE.name())); + // family not exists + assertThrows(NoSuchColumnFamilyException.class, + () -> ADMIN.modifyColumnFamilyStoreFileTracker(tableName, Bytes.toBytes("not_exists"), + StoreFileTrackerFactory.Trackers.FILE.name())); + // to migration + assertThrows(DoNotRetryIOException.class, () -> ADMIN.modifyTableStoreFileTracker(tableName, + StoreFileTrackerFactory.Trackers.MIGRATION.name())); + // disabled + ADMIN.disableTable(tableName); + assertThrows(TableNotEnabledException.class, () -> ADMIN.modifyTableStoreFileTracker(tableName, + StoreFileTrackerFactory.Trackers.FILE.name())); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminModifyStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminModifyStoreFileTracker.java new file mode 100644 index 000000000000..c8821b68cf57 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminModifyStoreFileTracker.java @@ -0,0 +1,197 @@ +/** + * 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.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; + +import java.io.IOException; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNotEnabledException; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; +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; +import org.apache.hadoop.hbase.util.FutureUtils; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +@Category({ LargeTests.class, ClientTests.class }) +public class TestAsyncAdminModifyStoreFileTracker extends TestAsyncAdminBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncAdminModifyStoreFileTracker.class); + + private static final String SRC_IMPL = "hbase.store.file-tracker.migration.src.impl"; + + private static final String DST_IMPL = "hbase.store.file-tracker.migration.dst.impl"; + + private void verifyModifyTableResult(TableName tableName, byte[] family, byte[] qual, byte[] row, + byte[] value, String sft) throws IOException { + TableDescriptor td = admin.getDescriptor(tableName).join(); + assertEquals(sft, td.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + // no migration related configs + assertNull(td.getValue(SRC_IMPL)); + assertNull(td.getValue(DST_IMPL)); + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual)); + } + } + + @Test + public void testModifyTableStoreFileTracker() throws IOException { + byte[] family = Bytes.toBytes("info"); + byte[] qual = Bytes.toBytes("q"); + byte[] row = Bytes.toBytes(0); + byte[] value = Bytes.toBytes(1); + try (Table table = TEST_UTIL.createTable(tableName, family)) { + table.put(new Put(row).addColumn(family, qual, value)); + } + // change to FILE + admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name()) + .join(); + verifyModifyTableResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to FILE again, should have no effect + admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name()) + .join(); + verifyModifyTableResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to MIGRATION, and then to FILE + admin.modifyTable(TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName).join()) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .setValue(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build()).join(); + admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name()) + .join(); + verifyModifyTableResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to MIGRATION, and then to DEFAULT + admin.modifyTable(TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName).join()) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setValue(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .setValue(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build()).join(); + admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .join(); + verifyModifyTableResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.DEFAULT.name()); + } + + private void verifyModifyColumnFamilyResult(TableName tableName, byte[] family, byte[] qual, + byte[] row, byte[] value, String sft) throws IOException { + TableDescriptor td = admin.getDescriptor(tableName).join(); + ColumnFamilyDescriptor cfd = td.getColumnFamily(family); + assertEquals(sft, cfd.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL)); + // no migration related configs + assertNull(cfd.getConfigurationValue(SRC_IMPL)); + assertNull(cfd.getConfigurationValue(DST_IMPL)); + assertNull(cfd.getValue(SRC_IMPL)); + assertNull(cfd.getValue(DST_IMPL)); + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual)); + } + } + + @Test + public void testModifyColumnFamilyStoreFileTracker() throws IOException { + byte[] family = Bytes.toBytes("info"); + byte[] qual = Bytes.toBytes("q"); + byte[] row = Bytes.toBytes(0); + byte[] value = Bytes.toBytes(1); + try (Table table = TEST_UTIL.createTable(tableName, family)) { + table.put(new Put(row).addColumn(family, qual, value)); + } + // change to FILE + admin.modifyColumnFamilyStoreFileTracker(tableName, family, + StoreFileTrackerFactory.Trackers.FILE.name()).join(); + verifyModifyColumnFamilyResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to FILE again, should have no effect + admin.modifyColumnFamilyStoreFileTracker(tableName, family, + StoreFileTrackerFactory.Trackers.FILE.name()).join(); + verifyModifyColumnFamilyResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to MIGRATION, and then to FILE + TableDescriptor current = admin.getDescriptor(tableName).join(); + admin.modifyTable(TableDescriptorBuilder.newBuilder(current) + .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family)) + .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build()) + .build()).join(); + admin.modifyColumnFamilyStoreFileTracker(tableName, family, + StoreFileTrackerFactory.Trackers.FILE.name()).join(); + verifyModifyColumnFamilyResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.FILE.name()); + + // change to MIGRATION, and then to DEFAULT + current = admin.getDescriptor(tableName).join(); + admin.modifyTable(TableDescriptorBuilder.newBuilder(current) + .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family)) + .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.MIGRATION.name()) + .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build()) + .build()).join(); + admin.modifyColumnFamilyStoreFileTracker(tableName, family, + StoreFileTrackerFactory.Trackers.DEFAULT.name()).join(); + verifyModifyColumnFamilyResult(tableName, family, qual, row, value, + StoreFileTrackerFactory.Trackers.DEFAULT.name()); + } + + @Test + public void testModifyStoreFileTrackerError() throws IOException { + byte[] family = Bytes.toBytes("info"); + TEST_UTIL.createTable(tableName, family).close(); + + // table not exists + assertThrows(TableNotFoundException.class, + () -> FutureUtils.get(admin.modifyTableStoreFileTracker(TableName.valueOf("whatever"), + StoreFileTrackerFactory.Trackers.FILE.name()))); + // family not exists + assertThrows(NoSuchColumnFamilyException.class, + () -> FutureUtils.get(admin.modifyColumnFamilyStoreFileTracker(tableName, + Bytes.toBytes("not_exists"), StoreFileTrackerFactory.Trackers.FILE.name()))); + // to migration + assertThrows(DoNotRetryIOException.class, () -> FutureUtils.get(admin + .modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.MIGRATION.name()))); + // disabled + admin.disableTable(tableName).join(); + assertThrows(TableNotEnabledException.class, () -> FutureUtils.get( + admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name()))); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java index e04f516559b3..5302cd59a97b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java @@ -498,4 +498,16 @@ public boolean normalizeRegions(NormalizeTableFilterParams ntfp, boolean isHighP public MetaLocationSyncer getMetaLocationSyncer() { return null; } + + @Override + public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup, + long nonce) throws IOException { + return -1; + } + + @Override + public long modifyColumnStoreFileTracker(TableName tableName, byte[] family, String dstSFT, + long nonceGroup, long nonce) throws IOException { + return -1; + } } 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 91038e9fe176..41f2afdfa421 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,16 +22,9 @@ 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; @@ -62,49 +55,4 @@ 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.validatePreRestoreSnapshot(td, snapTd, conf); - // removing cf level config is fine when it matches the td config - StoreFileTrackerFactory.validatePreRestoreSnapshot(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.validatePreRestoreSnapshot(td, defaultTd, conf); - }); - assertThrows(RestoreSnapshotException.class, () -> { - StoreFileTrackerFactory.validatePreRestoreSnapshot(snapTd, defaultTd, conf); - }); - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerValidationUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerValidationUtils.java new file mode 100644 index 000000000000..a686b559f9aa --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerValidationUtils.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 static org.junit.Assert.assertThrows; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +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.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; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestStoreFileTrackerValidationUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestStoreFileTrackerValidationUtils.class); + + @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 + StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(td, snapTd, conf); + // removing cf level config is fine when it matches the td config + StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(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, () -> { + StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(td, defaultTd, conf); + }); + assertThrows(RestoreSnapshotException.class, () -> { + StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(snapTd, defaultTd, conf); + }); + } +} 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 2a5d3370b61b..5950523f757c 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 @@ -1452,4 +1452,18 @@ public List getLogEntries(Set serverNames, String logType, throws IOException { throw new NotImplementedException("getLogEntries not supported in ThriftAdmin"); } + + @Override + public Future modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family, + String dstSFT) throws IOException { + throw new NotImplementedException( + "modifyColumnFamilyStoreFileTrackerAsync not supported in ThriftAdmin"); + } + + @Override + public Future modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT) + throws IOException { + throw new NotImplementedException( + "modifyTableStoreFileTrackerAsync not supported in ThriftAdmin"); + } } From 1d3d35cbe967c2b2dc2b1ec96e912f6c098b9cbf Mon Sep 17 00:00:00 2001 From: LiangJun He <2005hithlj@163.com> Date: Sat, 19 Feb 2022 23:00:04 +0800 Subject: [PATCH 26/32] HBASE-26673 Implement a shell command for change SFT implementation (#4113) Signed-off-by: Duo Zhang --- hbase-shell/src/main/ruby/hbase/admin.rb | 11 ++++ hbase-shell/src/main/ruby/shell.rb | 9 +++ .../main/ruby/shell/commands/change_sft.rb | 50 ++++++++++++++++ .../ruby/shell/commands/change_sft_all.rb | 58 +++++++++++++++++++ .../hbase/client/TestChangeSftShell.java | 46 +++++++++++++++ .../test/ruby/shell/sftchange_shell_test.rb | 56 ++++++++++++++++++ 6 files changed, 230 insertions(+) create mode 100644 hbase-shell/src/main/ruby/shell/commands/change_sft.rb create mode 100644 hbase-shell/src/main/ruby/shell/commands/change_sft_all.rb create mode 100644 hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestChangeSftShell.java create mode 100644 hbase-shell/src/test/ruby/shell/sftchange_shell_test.rb diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 25f07991a868..eab44920ce99 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -1789,6 +1789,17 @@ def to_server_names(server_names) end end + #---------------------------------------------------------------------------------------------- + # Change table's sft + def modify_table_sft(tableName, sft) + @admin.modifyTableStoreFileTracker(tableName, sft) + end + + #---------------------------------------------------------------------------------------------- + # Change table column family's sft + def modify_table_family_sft(tableName, family_bytes, sft) + @admin.modifyColumnFamilyStoreFileTracker(tableName, family_bytes, sft) + end end # rubocop:enable Metrics/ClassLength end diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb index a4bf49062a64..8b6c4f698b6e 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -624,3 +624,12 @@ def self.exception_handler(hide_traceback) get_namespace_rsgroup ] ) + +Shell.load_command_group( + 'storefiletracker', + full_name: 'StoreFileTracker', + commands: %w[ + change_sft + change_sft_all + ] +) diff --git a/hbase-shell/src/main/ruby/shell/commands/change_sft.rb b/hbase-shell/src/main/ruby/shell/commands/change_sft.rb new file mode 100644 index 000000000000..eb96f426a6a1 --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/change_sft.rb @@ -0,0 +1,50 @@ +# +# +# 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. +# + +module Shell + module Commands + class ChangeSft < Command + def help + <<-EOF +Change table's or table column family's sft. Examples: + + hbase> change_sft 't1','FILE' + hbase> change_sft 't2','cf1','FILE' +EOF + end + + def command(*args) + arg_length = args.length + if arg_length == 2 + tableName = TableName.valueOf(args[0]) + sft = args[1] + admin.modify_table_sft(tableName, sft) + elsif arg_length == 3 + tableName = TableName.valueOf(args[0]) + family = args[1] + family_bytes = family.to_java_bytes + sft = args[2] + admin.modify_table_family_sft(tableName, family_bytes, sft) + else + raise(ArgumentError, 'Argument length should be two or three.') + end + end + end + end +end diff --git a/hbase-shell/src/main/ruby/shell/commands/change_sft_all.rb b/hbase-shell/src/main/ruby/shell/commands/change_sft_all.rb new file mode 100644 index 000000000000..6e348195babe --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/change_sft_all.rb @@ -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. +# + +module Shell + module Commands + class ChangeSftAll < Command + def help + <<-EOF +Change all of the tables's sft matching the given regex: + + hbase> change_sft_all 't.*','FILE' + hbase> change_sft_all 'ns:.*','FILE' + hbase> change_sft_all 'ns:t.*','FILE' +EOF + end + + def command(*args) + arg_length = args.length + if arg_length == 2 + tableRegex = args[0] + tableList = admin.list(tableRegex) + count = tableList.size + sft = args[1] + tableList.each do |table| + formatter.row([table]) + end + puts "\nChange the above #{count} tables's sft (y/n)?" unless count == 0 + answer = 'n' + answer = gets.chomp unless count == 0 + puts "No tables matched the regex #{tableRegex}" if count == 0 + return unless answer =~ /y.*/i + tableList.each do |table| + tableName = TableName.valueOf(table) + admin.modify_table_sft(tableName, sft) + end + else + raise(ArgumentError, 'Argument length should be two.') + end + end + end + end +end diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestChangeSftShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestChangeSftShell.java new file mode 100644 index 000000000000..5e96e27bf1b1 --- /dev/null +++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestChangeSftShell.java @@ -0,0 +1,46 @@ +/** + * 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 org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; + +@Category({ ClientTests.class, LargeTests.class }) +public class TestChangeSftShell extends AbstractTestShell { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChangeSftShell.class); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + setUpConfig(); + + TEST_UTIL.startMiniCluster(3); + + setUpJRubyRuntime(); + } + + @Override + protected String getIncludeList() { + return "sftchange_shell_test.rb"; + } +} diff --git a/hbase-shell/src/test/ruby/shell/sftchange_shell_test.rb b/hbase-shell/src/test/ruby/shell/sftchange_shell_test.rb new file mode 100644 index 000000000000..220b624e2b47 --- /dev/null +++ b/hbase-shell/src/test/ruby/shell/sftchange_shell_test.rb @@ -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. +# + +require 'hbase_constants' +require 'hbase_shell' + +class SftChangeTest < Test::Unit::TestCase + def setup + @hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration) + @shell = Shell::Shell.new(@hbase) + connection = $TEST_CLUSTER.getConnection + @admin = connection.getAdmin + end + + define_test "Change table's sft" do + table = 'test_table1' + family = 'f1' + change_sft = 'FILE' + @shell.command('create', table, family) + @shell.command('change_sft', table, change_sft) + table_sft = @admin.getDescriptor(TableName.valueOf(table)).getValue('hbase.store.file-tracker.impl') + assert_equal(change_sft, table_sft) + @shell.command(:disable, table) + @shell.command(:drop, table) + end + + define_test "Change table column family's sft" do + table = 'test_table2' + family = 'f1' + change_sft = 'FILE' + @shell.command('create', table, family) + @shell.command('change_sft', table, family, change_sft) + family_bytes = family.to_java_bytes + cfd = @admin.getDescriptor(TableName.valueOf(table)).getColumnFamily(family_bytes) + table_family_sft = cfd.getConfigurationValue('hbase.store.file-tracker.impl') + assert_equal(change_sft, table_family_sft) + @shell.command(:disable, table) + @shell.command(:drop, table) + end +end From a3d1419bddabbd6aa7f762e2f527c093b8b86657 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 24 Feb 2022 10:10:02 +0800 Subject: [PATCH 27/32] HBASE-26640 Reimplement master local region initialization to better work with SFT (#4111) Signed-off-by: Josh Elser Signed-off-by: Wellington Chevreuil --- .../apache/hadoop/hbase/master/HMaster.java | 4 +- .../hbase/master/region/MasterRegion.java | 141 +++++++++++++++--- .../master/region/MasterRegionFactory.java | 22 ++- .../StoreFileTrackerFactory.java | 6 +- .../hadoop/hbase/util/FSTableDescriptors.java | 7 + .../master/region/MasterRegionTestBase.java | 16 +- .../region/TestChangeSFTForMasterRegion.java | 86 +++++++++++ .../region/TestMasterRegionInitialize.java | 117 +++++++++++++++ .../TestMasterRegionOnTwoFileSystems.java | 9 +- 9 files changed, 371 insertions(+), 37 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestChangeSFTForMasterRegion.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionInitialize.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 5d82a7c183a0..23dcfef95b81 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 @@ -781,7 +781,7 @@ private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperEx // done with in a one row put, which means if we have data in catalog family then we can // make sure that the migration is done. LOG.info("The {} family in master local region already has data in it, skip migrating...", - HConstants.CATALOG_FAMILY); + HConstants.CATALOG_FAMILY_STR); return; } } @@ -4081,7 +4081,7 @@ public MetaLocationSyncer getMetaLocationSyncer() { @RestrictedApi(explanation = "Should only be called in tests", link = "", allowedOnPath = ".*/src/test/.*") - MasterRegion getMasterRegion() { + public MasterRegion getMasterRegion() { return masterRegion; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java index 71950bd9167c..a0b8686882c5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.HConstants.HREGION_LOGDIR_NAME; import java.io.IOException; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -27,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -34,13 +36,18 @@ import org.apache.hadoop.hbase.client.ResultScanner; 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.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.RegionScanner; +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.wal.AbstractFSWAL; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.util.RecoverLeaseFSUtils; @@ -92,6 +99,10 @@ public final class MasterRegion { private static final String DEAD_WAL_DIR_SUFFIX = "-dead"; + static final String INITIALIZING_FLAG = ".initializing"; + + static final String INITIALIZED_FLAG = ".initialized"; + private static final int REGION_ID = 1; private final WALFactory walFactory; @@ -196,32 +207,39 @@ private static WAL createWAL(WALFactory walFactory, MasterRegionWALRoller walRol private static HRegion bootstrap(Configuration conf, TableDescriptor td, FileSystem fs, Path rootDir, FileSystem walFs, Path walRootDir, WALFactory walFactory, - MasterRegionWALRoller walRoller, String serverName) throws IOException { + MasterRegionWALRoller walRoller, String serverName, boolean touchInitializingFlag) + throws IOException { TableName tn = td.getTableName(); RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tn).setRegionId(REGION_ID).build(); - Path tmpTableDir = CommonFSUtils.getTableDir(rootDir, - TableName.valueOf(tn.getNamespaceAsString(), tn.getQualifierAsString() + "-tmp")); - if (fs.exists(tmpTableDir) && !fs.delete(tmpTableDir, true)) { - throw new IOException("Can not delete partial created proc region " + tmpTableDir); - } - HRegion.createHRegion(conf, regionInfo, fs, tmpTableDir, td).close(); Path tableDir = CommonFSUtils.getTableDir(rootDir, tn); - if (!fs.rename(tmpTableDir, tableDir)) { - throw new IOException("Can not rename " + tmpTableDir + " to " + tableDir); + // persist table descriptor + FSTableDescriptors.createTableDescriptorForTableDirectory(fs, tableDir, td, true); + HRegion.createHRegion(conf, regionInfo, fs, tableDir, td).close(); + Path initializedFlag = new Path(tableDir, INITIALIZED_FLAG); + if (!fs.mkdirs(initializedFlag)) { + throw new IOException("Can not touch initialized flag: " + initializedFlag); + } + Path initializingFlag = new Path(tableDir, INITIALIZING_FLAG); + if (!fs.delete(initializingFlag, true)) { + LOG.warn("failed to clean up initializing flag: " + initializingFlag); } WAL wal = createWAL(walFactory, walRoller, serverName, walFs, walRootDir, regionInfo); return HRegion.openHRegionFromTableDir(conf, fs, tableDir, regionInfo, td, wal, null, null); } - private static HRegion open(Configuration conf, TableDescriptor td, FileSystem fs, Path rootDir, - FileSystem walFs, Path walRootDir, WALFactory walFactory, MasterRegionWALRoller walRoller, - String serverName) throws IOException { - Path tableDir = CommonFSUtils.getTableDir(rootDir, td.getTableName()); - Path regionDir = - fs.listStatus(tableDir, p -> RegionInfo.isEncodedRegionName(Bytes.toBytes(p.getName())))[0] - .getPath(); - RegionInfo regionInfo = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); + private static RegionInfo loadRegionInfo(FileSystem fs, Path tableDir) throws IOException { + // on branch-2, the RegionInfo.isEncodedRegionName will returns true for .initializing and + // .initialized, see HBASE-25368. Since RegionInfo is IA.Public, changing the implementation may + // raise compatibility concerns, so here we just skip them by our own. + Path regionDir = fs.listStatus(tableDir, p -> !p.getName().startsWith(".") + && RegionInfo.isEncodedRegionName(Bytes.toBytes(p.getName())))[0].getPath(); + return HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); + } + private static HRegion open(Configuration conf, TableDescriptor td, RegionInfo regionInfo, + FileSystem fs, Path rootDir, FileSystem walFs, Path walRootDir, WALFactory walFactory, + MasterRegionWALRoller walRoller, String serverName) throws IOException { + Path tableDir = CommonFSUtils.getTableDir(rootDir, td.getTableName()); Path walRegionDir = FSUtils.getRegionDirFromRootDir(walRootDir, regionInfo); Path replayEditsDir = new Path(walRegionDir, REPLAY_EDITS_DIR); if (!walFs.exists(replayEditsDir) && !walFs.mkdirs(replayEditsDir)) { @@ -287,6 +305,39 @@ private static void replayWALs(Configuration conf, FileSystem walFs, Path walRoo } } + private static void tryMigrate(Configuration conf, FileSystem fs, Path tableDir, + RegionInfo regionInfo, TableDescriptor oldTd, TableDescriptor newTd) throws IOException { + Class oldSft = + StoreFileTrackerFactory.getTrackerClass(oldTd.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + Class newSft = + StoreFileTrackerFactory.getTrackerClass(newTd.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + if (oldSft.equals(newSft)) { + LOG.debug("old store file tracker {} is the same with new store file tracker, skip migration", + StoreFileTrackerFactory.getStoreFileTrackerName(oldSft)); + if (!oldTd.equals(newTd)) { + // we may change other things such as adding a new family, so here we still need to persist + // the new table descriptor + LOG.info("Update table descriptor from {} to {}", oldTd, newTd); + FSTableDescriptors.createTableDescriptorForTableDirectory(fs, tableDir, newTd, true); + } + return; + } + LOG.info("Migrate store file tracker from {} to {}", oldSft.getSimpleName(), + newSft.getSimpleName()); + HRegionFileSystem hfs = + HRegionFileSystem.openRegionFromFileSystem(conf, fs, tableDir, regionInfo, false); + for (ColumnFamilyDescriptor oldCfd : oldTd.getColumnFamilies()) { + StoreFileTracker oldTracker = StoreFileTrackerFactory.create(conf, oldTd, oldCfd, hfs); + StoreFileTracker newTracker = StoreFileTrackerFactory.create(conf, oldTd, oldCfd, hfs); + List files = oldTracker.load(); + LOG.debug("Store file list for {}: {}", oldCfd.getNameAsString(), files); + newTracker.set(oldTracker.load()); + } + // persist the new table descriptor after migration + LOG.info("Update table descriptor from {} to {}", oldTd, newTd); + FSTableDescriptors.createTableDescriptorForTableDirectory(fs, tableDir, newTd, true); + } + public static MasterRegion create(MasterRegionParams params) throws IOException { TableDescriptor td = params.tableDescriptor(); LOG.info("Create or load local region for table " + td); @@ -321,16 +372,58 @@ public static MasterRegion create(MasterRegionParams params) throws IOException WALFactory walFactory = new WALFactory(conf, server.getServerName().toString()); Path tableDir = CommonFSUtils.getTableDir(rootDir, td.getTableName()); + Path initializingFlag = new Path(tableDir, INITIALIZING_FLAG); + Path initializedFlag = new Path(tableDir, INITIALIZED_FLAG); HRegion region; - if (fs.exists(tableDir)) { - // load the existing region. - region = open(conf, td, fs, rootDir, walFs, walRootDir, walFactory, walRoller, - server.getServerName().toString()); - } else { - // bootstrapping... + if (!fs.exists(tableDir)) { + // bootstrap, no doubt + if (!fs.mkdirs(initializedFlag)) { + throw new IOException("Can not touch initialized flag"); + } region = bootstrap(conf, td, fs, rootDir, walFs, walRootDir, walFactory, walRoller, - server.getServerName().toString()); + server.getServerName().toString(), true); + } else { + if (!fs.exists(initializedFlag)) { + if (!fs.exists(initializingFlag)) { + // should be old style, where we do not have the initializing or initialized file, persist + // the table descriptor, touch the initialized flag and then open the region. + // the store file tracker must be DEFAULT + LOG.info("No {} or {} file, try upgrading", INITIALIZING_FLAG, INITIALIZED_FLAG); + TableDescriptor oldTd = + TableDescriptorBuilder.newBuilder(td).setValue(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.DEFAULT.name()).build(); + FSTableDescriptors.createTableDescriptorForTableDirectory(fs, tableDir, oldTd, true); + if (!fs.mkdirs(initializedFlag)) { + throw new IOException("Can not touch initialized flag: " + initializedFlag); + } + RegionInfo regionInfo = loadRegionInfo(fs, tableDir); + tryMigrate(conf, fs, tableDir, regionInfo, oldTd, td); + region = open(conf, td, regionInfo, fs, rootDir, walFs, walRootDir, walFactory, walRoller, + server.getServerName().toString()); + } else { + // delete all contents besides the initializing flag, here we can make sure tableDir + // exists(unless someone delete it manually...), so we do not do null check here. + for (FileStatus status : fs.listStatus(tableDir)) { + if (!status.getPath().getName().equals(INITIALIZING_FLAG)) { + fs.delete(status.getPath(), true); + } + } + region = bootstrap(conf, td, fs, rootDir, walFs, walRootDir, walFactory, walRoller, + server.getServerName().toString(), false); + } + } else { + if (fs.exists(initializingFlag) && !fs.delete(initializingFlag, true)) { + LOG.warn("failed to clean up initializing flag: " + initializingFlag); + } + // open it, make sure to load the table descriptor from fs + TableDescriptor oldTd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); + RegionInfo regionInfo = loadRegionInfo(fs, tableDir); + tryMigrate(conf, fs, tableDir, regionInfo, oldTd, td); + region = open(conf, td, regionInfo, fs, rootDir, walFs, walRootDir, walFactory, walRoller, + server.getServerName().toString()); + } } + Path globalArchiveDir = HFileArchiveUtil.getArchivePath(baseConf); MasterRegionFlusherAndCompactor flusherAndCompactor = new MasterRegionFlusherAndCompactor(conf, server, region, params.flushSize(), params.flushPerChanges(), params.flushIntervalMs(), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegionFactory.java index d70aef6af170..3ef6224c4d9b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegionFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegionFactory.java @@ -28,7 +28,10 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.regionserver.BloomType; +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.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; /** @@ -78,6 +81,8 @@ public final class MasterRegionFactory { private static final int DEFAULT_RING_BUFFER_SLOT_COUNT = 128; + public static final String TRACKER_IMPL = "hbase.master.store.region.file-tracker.impl"; + public static final TableName TABLE_NAME = TableName.valueOf("master:store"); public static final byte[] PROC_FAMILY = Bytes.toBytes("proc"); @@ -89,10 +94,23 @@ public final class MasterRegionFactory { .setDataBlockEncoding(DataBlockEncoding.ROW_INDEX_V1).build()) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(PROC_FAMILY)).build(); + private static TableDescriptor withTrackerConfigs(Configuration conf) { + String trackerImpl = conf.get(TRACKER_IMPL, conf.get(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.DEFAULT.name())); + Class trackerClass = + StoreFileTrackerFactory.getTrackerClass(trackerImpl); + if (StoreFileTrackerFactory.isMigration(trackerClass)) { + throw new IllegalArgumentException("Should not set store file tracker to " + + StoreFileTrackerFactory.Trackers.MIGRATION.name() + " for master local region"); + } + StoreFileTracker tracker = ReflectionUtils.newInstance(trackerClass, conf, true, null); + return tracker.updateWithTrackerConfigs(TableDescriptorBuilder.newBuilder(TABLE_DESC)).build(); + } + public static MasterRegion create(Server server) throws IOException { - MasterRegionParams params = new MasterRegionParams().server(server) - .regionDirName(MASTER_STORE_DIR).tableDescriptor(TABLE_DESC); Configuration conf = server.getConfiguration(); + MasterRegionParams params = new MasterRegionParams().server(server) + .regionDirName(MASTER_STORE_DIR).tableDescriptor(withTrackerConfigs(conf)); long flushSize = conf.getLong(FLUSH_SIZE_KEY, DEFAULT_FLUSH_SIZE); long flushPerChanges = conf.getLong(FLUSH_PER_CHANGES_KEY, DEFAULT_FLUSH_PER_CHANGES); long flushIntervalMs = conf.getLong(FLUSH_INTERVAL_MS_KEY, DEFAULT_FLUSH_INTERVAL_MS); 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 7f545e4058d7..85c5ee24f3b2 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 @@ -84,7 +84,7 @@ public static String getStoreFileTrackerName(Configuration conf) { return conf.get(TRACKER_IMPL, Trackers.DEFAULT.name()); } - static String getStoreFileTrackerName(Class clazz) { + public static String getStoreFileTrackerName(Class clazz) { Trackers name = CLASS_TO_ENUM.get(clazz); return name != null ? name.name() : clazz.getName(); } @@ -184,4 +184,8 @@ public static TableDescriptor updateWithTrackerConfigs(Configuration conf, } return descriptor; } + + public static boolean isMigration(Class clazz) { + return MigrationStoreFileTracker.class.isAssignableFrom(clazz); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 8b854053444f..ff555a8c40d6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -511,6 +511,13 @@ private static Optional> getTableDescriptorFro return td != null ? Optional.of(Pair.newPair(descFile, td)) : Optional.empty(); } + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*/src/test/.*") + public static void deleteTableDescriptors(FileSystem fs, Path tableDir) throws IOException { + Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR); + deleteTableDescriptorFiles(fs, tableInfoDir, Integer.MAX_VALUE); + } + /** * Deletes files matching the table info file pattern within the given directory whose sequenceId * is at most the given max sequenceId. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/MasterRegionTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/MasterRegionTestBase.java index 27a4b7a96329..0d35fdd7cea4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/MasterRegionTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/MasterRegionTestBase.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.cleaner.DirScanPool; import org.apache.hadoop.hbase.regionserver.MemStoreLAB; +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; @@ -85,19 +86,24 @@ public void setUp() throws IOException { /** * Creates a new MasterRegion using an existing {@code htu} on this class. */ - protected void createMasterRegion() throws IOException { - configure(htu.getConfiguration()); + protected final void createMasterRegion() throws IOException { + Configuration conf = htu.getConfiguration(); + configure(conf); choreService = new ChoreService(getClass().getSimpleName()); cleanerPool = new DirScanPool(htu.getConfiguration()); Server server = mock(Server.class); - when(server.getConfiguration()).thenReturn(htu.getConfiguration()); + when(server.getConfiguration()).thenReturn(conf); when(server.getServerName()) .thenReturn(ServerName.valueOf("localhost", 12345, EnvironmentEdgeManager.currentTime())); when(server.getChoreService()).thenReturn(choreService); Path testDir = htu.getDataTestDir(); - CommonFSUtils.setRootDir(htu.getConfiguration(), testDir); + CommonFSUtils.setRootDir(conf, testDir); MasterRegionParams params = new MasterRegionParams(); - params.server(server).regionDirName(REGION_DIR_NAME).tableDescriptor(TD) + TableDescriptor td = TableDescriptorBuilder + .newBuilder(TD).setValue(StoreFileTrackerFactory.TRACKER_IMPL, conf + .get(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())) + .build(); + params.server(server).regionDirName(REGION_DIR_NAME).tableDescriptor(td) .flushSize(TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE).flushPerChanges(1_000_000) .flushIntervalMs(TimeUnit.MINUTES.toMillis(15)).compactMin(4).maxWals(32).useHsync(false) .ringBufferSlotCount(16).rollPeriodMs(TimeUnit.MINUTES.toMillis(15)) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestChangeSFTForMasterRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestChangeSFTForMasterRegion.java new file mode 100644 index 000000000000..3f7c7a8f19cb --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestChangeSFTForMasterRegion.java @@ -0,0 +1,86 @@ +/** + * 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.region; + +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Make sure we do not loss data after changing SFT implementation + */ +@Category({ MasterTests.class, MediumTests.class }) +public class TestChangeSFTForMasterRegion { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChangeSFTForMasterRegion.class); + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + private static TableName NAME = TableName.valueOf("test"); + + private static byte[] FAMILY = Bytes.toBytes("family"); + + @BeforeClass + public static void setUp() throws Exception { + UTIL.getConfiguration().set(MasterRegionFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.DEFAULT.name()); + // use zk connection registry, as we will shutdown the only master instance which will likely to + // lead to dead loop + UTIL.getConfiguration().set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + HConstants.ZK_CONNECTION_REGISTRY_CLASS); + UTIL.startMiniCluster(1); + UTIL.createTable(NAME, FAMILY).close(); + } + + @AfterClass + public static void tearDown() throws IOException { + UTIL.shutdownMiniCluster(); + } + + @Test + public void test() throws Exception { + // shutdown master + UTIL.getMiniHBaseCluster().stopMaster(0).join(); + UTIL.getMiniHBaseCluster().getConf().set(MasterRegionFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.FILE.name()); + UTIL.getMiniHBaseCluster().startMaster(); + // make sure that the table still exists + UTIL.waitTableAvailable(NAME); + // confirm that we have changed the SFT to FILE + TableDescriptor td = + UTIL.getMiniHBaseCluster().getMaster().getMasterRegion().region.getTableDescriptor(); + assertEquals(StoreFileTrackerFactory.Trackers.FILE.name(), + td.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionInitialize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionInitialize.java new file mode 100644 index 000000000000..5ebde72db5d0 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionInitialize.java @@ -0,0 +1,117 @@ +/** + * 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.region; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +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.FSTableDescriptors; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MasterTests.class, MediumTests.class }) +public class TestMasterRegionInitialize extends MasterRegionTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterRegionInitialize.class); + + @Test + public void testUpgrade() throws IOException { + Path rootDir = new Path(htu.getDataTestDir(), REGION_DIR_NAME); + Path tableDir = + CommonFSUtils.getTableDir(rootDir, region.region.getTableDescriptor().getTableName()); + Path initializingFlag = new Path(tableDir, MasterRegion.INITIALIZING_FLAG); + Path initializedFlag = new Path(tableDir, MasterRegion.INITIALIZED_FLAG); + HRegionFileSystem hfs = region.region.getRegionFileSystem(); + assertFalse(hfs.getFileSystem().exists(initializingFlag)); + assertTrue(hfs.getFileSystem().exists(initializedFlag)); + byte[] row = Bytes.toBytes("row"); + byte[] cf = CF1; + byte[] cq = Bytes.toBytes("qual"); + byte[] value = Bytes.toBytes("value"); + region.update(r -> r.put(new Put(row).addColumn(cf, cq, value))); + assertEquals(FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, region.flush(true).getResult()); + // delete initialized flag to simulate old implementation + hfs.getFileSystem().delete(initializedFlag, true); + FSTableDescriptors.deleteTableDescriptors(hfs.getFileSystem(), tableDir); + assertNull(FSTableDescriptors.getTableDescriptorFromFs(hfs.getFileSystem(), tableDir)); + // reopen, with new file tracker + region.close(false); + htu.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.FILE.name()); + createMasterRegion(); + + // make sure we successfully upgrade to new implementation without data loss + hfs = region.region.getRegionFileSystem(); + assertFalse(hfs.getFileSystem().exists(initializingFlag)); + assertTrue(hfs.getFileSystem().exists(initializedFlag)); + TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(hfs.getFileSystem(), tableDir); + assertEquals(StoreFileTrackerFactory.Trackers.FILE.name(), + td.getValue(StoreFileTrackerFactory.TRACKER_IMPL)); + assertArrayEquals(value, region.get(new Get(row)).getValue(cf, cq)); + } + + @Test + public void testInitializingCleanup() throws IOException { + Path rootDir = new Path(htu.getDataTestDir(), REGION_DIR_NAME); + Path tableDir = + CommonFSUtils.getTableDir(rootDir, region.region.getTableDescriptor().getTableName()); + Path initializingFlag = new Path(tableDir, MasterRegion.INITIALIZING_FLAG); + Path initializedFlag = new Path(tableDir, MasterRegion.INITIALIZED_FLAG); + HRegionFileSystem hfs = region.region.getRegionFileSystem(); + assertFalse(hfs.getFileSystem().exists(initializingFlag)); + assertTrue(hfs.getFileSystem().exists(initializedFlag)); + byte[] row = Bytes.toBytes("row"); + byte[] cf = CF1; + byte[] cq = Bytes.toBytes("qual"); + byte[] value = Bytes.toBytes("value"); + region.update(r -> r.put(new Put(row).addColumn(cf, cq, value))); + // delete initialized flag and touch a initializing flag, to simulate initializing in progress + hfs.getFileSystem().delete(initializedFlag, true); + if (!hfs.getFileSystem().mkdirs(initializingFlag)) { + throw new IOException("can not touch " + initializedFlag); + } + + region.close(false); + createMasterRegion(); + hfs = region.region.getRegionFileSystem(); + assertFalse(hfs.getFileSystem().exists(initializingFlag)); + assertTrue(hfs.getFileSystem().exists(initializedFlag)); + + // but the data should have been cleaned up + assertTrue(region.get(new Get(row)).isEmpty()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionOnTwoFileSystems.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionOnTwoFileSystems.java index 3665ffdc8f33..6bf8379cf433 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionOnTwoFileSystems.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionOnTwoFileSystems.java @@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.MemStoreLAB; import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -86,9 +87,11 @@ public class TestMasterRegionOnTwoFileSystems { private static byte[] CQ = Bytes.toBytes("q"); - private static TableDescriptor TD = - TableDescriptorBuilder.newBuilder(TableName.valueOf("test:local")) - .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build(); + private static TableDescriptor TD = TableDescriptorBuilder + .newBuilder(TableName.valueOf("test:local")) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()) + .build(); private static int COMPACT_MIN = 4; From 44d49993d707f93844323063e4f70c726f7a59f4 Mon Sep 17 00:00:00 2001 From: BukrosSzabolcs Date: Fri, 25 Feb 2022 21:11:41 +0100 Subject: [PATCH 28/32] HBASE-26707: Reduce number of renames during bulkload (#4066) (#4122) Signed-off-by: Wellington Ramos Chevreuil Conflicts: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java --- .../IntegrationTestFileBasedSFTBulkLoad.java | 108 +++++++ .../hadoop/hbase/regionserver/HRegion.java | 19 +- .../hbase/regionserver/HRegionFileSystem.java | 4 + .../regionserver/SecureBulkLoadManager.java | 64 +++-- .../hbase/regionserver/TestBulkLoad.java | 4 + .../hbase/regionserver/TestBulkloadBase.java | 36 ++- .../TestCompactionAfterBulkLoad.java | 4 + .../TestSecureBulkLoadManager.java | 39 ++- .../TestSecureBulkloadListener.java | 264 ++++++++++++++++++ 9 files changed, 503 insertions(+), 39 deletions(-) create mode 100644 hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestFileBasedSFTBulkLoad.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkloadListener.java diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestFileBasedSFTBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestFileBasedSFTBulkLoad.java new file mode 100644 index 000000000000..93e51fff8aab --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestFileBasedSFTBulkLoad.java @@ -0,0 +1,108 @@ +/** + * + * 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.mapreduce; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.IntegrationTestingUtility; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test Bulk Load and MR on a distributed cluster. + * With FileBased StorefileTracker enabled. + * It starts an MR job that creates linked chains + * + * The format of rows is like this: + * Row Key -> Long + * + * L:<< Chain Id >> -> Row Key of the next link in the chain + * S:<< Chain Id >> -> The step in the chain that his link is. + * D:<< Chain Id >> -> Random Data. + * + * All chains start on row 0. + * All rk's are > 0. + * + * After creating the linked lists they are walked over using a TableMapper based Mapreduce Job. + * + * There are a few options exposed: + * + * hbase.IntegrationTestBulkLoad.chainLength + * The number of rows that will be part of each and every chain. + * + * hbase.IntegrationTestBulkLoad.numMaps + * The number of mappers that will be run. Each mapper creates on linked list chain. + * + * hbase.IntegrationTestBulkLoad.numImportRounds + * How many jobs will be run to create linked lists. + * + * hbase.IntegrationTestBulkLoad.tableName + * The name of the table. + * + * hbase.IntegrationTestBulkLoad.replicaCount + * How many region replicas to configure for the table under test. + */ +@Category(IntegrationTests.class) +public class IntegrationTestFileBasedSFTBulkLoad extends IntegrationTestBulkLoad { + + private static final Logger LOG = LoggerFactory.getLogger(IntegrationTestFileBasedSFTBulkLoad.class); + + private static String NUM_MAPS_KEY = "hbase.IntegrationTestBulkLoad.numMaps"; + private static String NUM_IMPORT_ROUNDS_KEY = "hbase.IntegrationTestBulkLoad.numImportRounds"; + private static String NUM_REPLICA_COUNT_KEY = "hbase.IntegrationTestBulkLoad.replicaCount"; + private static int NUM_REPLICA_COUNT_DEFAULT = 1; + + @Test + public void testFileBasedSFTBulkLoad() throws Exception { + super.testBulkLoad(); + } + + @Override + public void setUpCluster() throws Exception { + util = getTestingUtil(getConf()); + util.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, + "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker"); + util.initializeCluster(1); + int replicaCount = getConf().getInt(NUM_REPLICA_COUNT_KEY, NUM_REPLICA_COUNT_DEFAULT); + if (LOG.isDebugEnabled() && replicaCount != NUM_REPLICA_COUNT_DEFAULT) { + LOG.debug("Region Replicas enabled: " + replicaCount); + } + + // Scale this up on a real cluster + if (util.isDistributedCluster()) { + util.getConfiguration().setIfUnset(NUM_MAPS_KEY, + Integer.toString(util.getAdmin().getRegionServers().size() * 10)); + util.getConfiguration().setIfUnset(NUM_IMPORT_ROUNDS_KEY, "5"); + } else { + util.startMiniMapReduceCluster(); + } + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + IntegrationTestingUtility.setUseDistributedCluster(conf); + int status = ToolRunner.run(conf, new IntegrationTestFileBasedSFTBulkLoad(), args); + System.exit(status); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 6745e5a7ad85..a17fbc84be60 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -6847,7 +6847,7 @@ public interface BulkLoadListener { * @return final path to be used for actual loading * @throws IOException */ - String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile) + String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile, String customStaging) throws IOException; /** @@ -6969,12 +6969,21 @@ public Map> bulkLoadHFiles(Collection> f familyWithFinalPath.put(familyName, new ArrayList<>()); } List> lst = familyWithFinalPath.get(familyName); + String finalPath = path; try { - String finalPath = path; + boolean reqTmp = store.storeEngine.requireWritingToTmpDirFirst(); if (bulkLoadListener != null) { - finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile); + finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile, + reqTmp ? null : regionDir.toString()); + } + Pair pair = null; + if (reqTmp) { + pair = store.preBulkLoadHFile(finalPath, seqId); + } + else { + Path livePath = new Path(finalPath); + pair = new Pair<>(livePath, livePath); } - Pair pair = store.preBulkLoadHFile(finalPath, seqId); lst.add(pair); } catch (IOException ioe) { // A failure here can cause an atomicity violation that we currently @@ -6984,7 +6993,7 @@ public Map> bulkLoadHFiles(Collection> f " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe); if (bulkLoadListener != null) { try { - bulkLoadListener.failedBulkLoad(familyName, path); + bulkLoadListener.failedBulkLoad(familyName, finalPath); } catch (Exception ex) { LOG.error("Error while calling failedBulkLoad for family " + Bytes.toString(familyName) + " with path " + path, ex); 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 8920471a86ee..3ed7b60b82b6 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 @@ -508,6 +508,10 @@ private Path preCommitStoreFile(final String familyName, final Path buildPath, * @throws IOException */ Path commitStoreFile(final Path buildPath, Path dstPath) throws IOException { + // rename is not necessary in case of direct-insert stores + if(buildPath.equals(dstPath)){ + return dstPath; + } // buildPath exists, therefore not doing an exists() check. if (!rename(buildPath, dstPath)) { throw new IOException("Failed rename of " + buildPath + " to " + dstPath); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java index f74e2f89be3e..dbc5f72e37b6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java @@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.function.Consumer; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -342,7 +343,8 @@ private User getActiveUser() throws IOException { return user; } - private static class SecureBulkLoadListener implements BulkLoadListener { + //package-private for test purpose only + static class SecureBulkLoadListener implements BulkLoadListener { // Target filesystem private final FileSystem fs; private final String stagingDir; @@ -350,19 +352,28 @@ private static class SecureBulkLoadListener implements BulkLoadListener { // Source filesystem private FileSystem srcFs = null; private Map origPermissions = null; + private Map origSources = null; public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) { this.fs = fs; this.stagingDir = stagingDir; this.conf = conf; this.origPermissions = new HashMap<>(); + this.origSources = new HashMap<>(); } @Override - public String prepareBulkLoad(final byte[] family, final String srcPath, boolean copyFile) - throws IOException { + public String prepareBulkLoad(final byte[] family, final String srcPath, boolean copyFile, + String customStaging ) throws IOException { Path p = new Path(srcPath); - Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName())); + + //store customStaging for failedBulkLoad + String currentStaging = stagingDir; + if(StringUtils.isNotEmpty(customStaging)){ + currentStaging = customStaging; + } + + Path stageP = new Path(currentStaging, new Path(Bytes.toString(family), p.getName())); // In case of Replication for bulk load files, hfiles are already copied in staging directory if (p.equals(stageP)) { @@ -391,11 +402,16 @@ public String prepareBulkLoad(final byte[] family, final String srcPath, boolean LOG.debug("Moving " + p + " to " + stageP); FileStatus origFileStatus = fs.getFileStatus(p); origPermissions.put(srcPath, origFileStatus.getPermission()); + origSources.put(stageP.toString(), srcPath); if(!fs.rename(p, stageP)) { throw new IOException("Failed to move HFile: " + p + " to " + stageP); } } - fs.setPermission(stageP, PERM_ALL_ACCESS); + + if(StringUtils.isNotEmpty(customStaging)) { + fs.setPermission(stageP, PERM_ALL_ACCESS); + } + return stageP.toString(); } @@ -413,35 +429,37 @@ private void closeSrcFs() throws IOException { } @Override - public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException { + public void failedBulkLoad(final byte[] family, final String stagedPath) throws IOException { try { - Path p = new Path(srcPath); - if (srcFs == null) { - srcFs = FileSystem.newInstance(p.toUri(), conf); - } - if (!FSUtils.isSameHdfs(conf, srcFs, fs)) { - // files are copied so no need to move them back + String src = origSources.get(stagedPath); + if(StringUtils.isEmpty(src)){ + LOG.debug(stagedPath + " was not moved to staging. No need to move back"); return; } - Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName())); - // In case of Replication for bulk load files, hfiles are not renamed by end point during - // prepare stage, so no need of rename here again - if (p.equals(stageP)) { - LOG.debug(p.getName() + " is already available in source directory. Skipping rename."); + Path stageP = new Path(stagedPath); + if (!fs.exists(stageP)) { + throw new IOException( + "Missing HFile: " + stageP + ", can't be moved back to it's original place"); + } + + //we should not move back files if the original exists + Path srcPath = new Path(src); + if(srcFs.exists(srcPath)) { + LOG.debug(src + " is already at it's original place. No need to move."); return; } - LOG.debug("Moving " + stageP + " back to " + p); - if (!fs.rename(stageP, p)) { - throw new IOException("Failed to move HFile: " + stageP + " to " + p); + LOG.debug("Moving " + stageP + " back to " + srcPath); + if (!fs.rename(stageP, srcPath)) { + throw new IOException("Failed to move HFile: " + stageP + " to " + srcPath); } // restore original permission - if (origPermissions.containsKey(srcPath)) { - fs.setPermission(p, origPermissions.get(srcPath)); + if (origPermissions.containsKey(stagedPath)) { + fs.setPermission(srcPath, origPermissions.get(src)); } else { - LOG.warn("Can't find previous permission for path=" + srcPath); + LOG.warn("Can't find previous permission for path=" + stagedPath); } } finally { closeSrcFs(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java index 3a934b749358..e7f168f1e02e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java @@ -54,6 +54,10 @@ public class TestBulkLoad extends TestBulkloadBase { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestBulkLoad.class); + public TestBulkLoad(boolean useFileBasedSFT) { + super(useFileBasedSFT); + } + @Test public void verifyBulkLoadEvent() throws IOException { TableName tableName = TableName.valueOf("test", "test"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java index 0db15e750d61..86e41ede8294 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java @@ -27,7 +27,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.List; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -44,6 +46,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WAL; @@ -56,10 +59,12 @@ import org.junit.Rule; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +@RunWith(Parameterized.class) public class TestBulkloadBase { @ClassRule public static TemporaryFolder testFolder = new TemporaryFolder(); @@ -71,12 +76,31 @@ public class TestBulkloadBase { protected final byte[] family2 = Bytes.toBytes("family2"); protected final byte[] family3 = Bytes.toBytes("family3"); + protected Boolean useFileBasedSFT; + @Rule public TestName name = new TestName(); + public TestBulkloadBase(boolean useFileBasedSFT) { + this.useFileBasedSFT = useFileBasedSFT; + } + + @Parameterized.Parameters + public static Collection data() { + Boolean[] data = {false, true}; + return Arrays.asList(data); + } + @Before public void before() throws IOException { Bytes.random(randomBytes); + if(useFileBasedSFT) { + conf.set(StoreFileTrackerFactory.TRACKER_IMPL, + "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker"); + } + else { + conf.unset(StoreFileTrackerFactory.TRACKER_IMPL); + } } protected Pair withMissingHFileForFamily(byte[] family) { @@ -111,7 +135,7 @@ protected HRegion testRegionWithFamiliesAndSpecifiedTableName(TableName tableNam } protected HRegion testRegionWithFamilies(byte[]... families) throws IOException { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = TableName.valueOf(name.getMethodName().substring(0, name.getMethodName().indexOf("["))); return testRegionWithFamiliesAndSpecifiedTableName(tableName, families); } @@ -130,7 +154,7 @@ protected List> withFamilyPathsFor(byte[]... families) thro private String createHFileForFamilies(byte[] family) throws IOException { HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf); // TODO We need a way to do this without creating files - File hFileLocation = testFolder.newFile(); + File hFileLocation = testFolder.newFile(generateUniqueName(null)); FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation), null); try { hFileFactory.withOutputStream(out); @@ -149,6 +173,12 @@ private String createHFileForFamilies(byte[] family) throws IOException { return hFileLocation.getAbsoluteFile().getAbsolutePath(); } + private static String generateUniqueName(final String suffix) { + String name = UUID.randomUUID().toString().replaceAll("-", ""); + if (suffix != null) name += suffix; + return name; + } + protected static Matcher bulkLogWalEditType(byte[] typeBytes) { return new WalMatcher(typeBytes); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java index b17995a591e4..70b81a3bc7df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java @@ -61,6 +61,10 @@ public class TestCompactionAfterBulkLoad extends TestBulkloadBase { private final RegionServerServices regionServerServices = mock(RegionServerServices.class); public static AtomicInteger called = new AtomicInteger(0); + public TestCompactionAfterBulkLoad(boolean useFileBasedSFT) { + super(useFileBasedSFT); + } + @Override protected HRegion testRegionWithFamiliesAndSpecifiedTableName(TableName tableName, byte[]... families) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java index b8ca951c355f..5ba95c3fb3d7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java @@ -19,12 +19,13 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; import java.util.Deque; import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -46,23 +47,26 @@ 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.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; -import org.junit.AfterClass; +import org.junit.After; import org.junit.Assert; -import org.junit.BeforeClass; +import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; - +@RunWith(Parameterized.class) @Category({RegionServerTests.class, MediumTests.class}) public class TestSecureBulkLoadManager { @@ -87,15 +91,34 @@ public class TestSecureBulkLoadManager { private Thread laterBulkload; protected final static HBaseTestingUtility testUtil = new HBaseTestingUtility(); + protected Boolean useFileBasedSFT; + private static Configuration conf = testUtil.getConfiguration(); - @BeforeClass - public static void setUp() throws Exception { + public TestSecureBulkLoadManager(Boolean useFileBasedSFT) { + this.useFileBasedSFT = useFileBasedSFT; + } + + @Parameterized.Parameters + public static Collection data() { + Boolean[] data = {false, true}; + return Arrays.asList(data); + } + + @Before + public void setUp() throws Exception { + if (useFileBasedSFT) { + conf.set(StoreFileTrackerFactory.TRACKER_IMPL, + "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker"); + } + else{ + conf.unset(StoreFileTrackerFactory.TRACKER_IMPL); + } testUtil.startMiniCluster(); } - @AfterClass - public static void tearDown() throws Exception { + @After + public void tearDown() throws Exception { testUtil.shutdownMiniCluster(); testUtil.cleanupTestDir(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkloadListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkloadListener.java new file mode 100644 index 000000000000..bf3732a821da --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkloadListener.java @@ -0,0 +1,264 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellBuilderType; +import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.After; +import org.junit.Assert; +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.TemporaryFolder; +import org.junit.rules.TestName; +import java.io.IOException; +import java.util.Random; +import java.util.UUID; + +/** + * Tests for failedBulkLoad logic to make sure staged files are returned to their original location + * if the bulkload have failed. + */ +@Category({MiscTests.class, LargeTests.class}) +public class TestSecureBulkloadListener { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecureBulkloadListener.class); + + @ClassRule + public static TemporaryFolder testFolder = new TemporaryFolder(); + private Configuration conf; + private MiniDFSCluster cluster; + private HBaseTestingUtility htu; + private DistributedFileSystem dfs; + private final Random random = new Random(); + private final byte[] randomBytes = new byte[100]; + private static final String host1 = "host1"; + private static final String host2 = "host2"; + private static final String host3 = "host3"; + private static byte[] FAMILY = Bytes.toBytes("family"); + private static final String STAGING_DIR = "staging"; + private static final String CUSTOM_STAGING_DIR = "customStaging"; + + @Rule + public TestName name = new TestName(); + + @Before + public void setUp() throws Exception { + random.nextBytes(randomBytes); + htu = new HBaseTestingUtility(); + htu.getConfiguration().setInt("dfs.blocksize", 1024);// For the test with multiple blocks + htu.getConfiguration().setInt("dfs.replication", 3); + htu.startMiniDFSCluster(3, + new String[]{"/r1", "/r2", "/r3"}, new String[]{host1, host2, host3}); + + conf = htu.getConfiguration(); + cluster = htu.getDFSCluster(); + dfs = (DistributedFileSystem) FileSystem.get(conf); + } + + @After + public void tearDownAfterClass() throws Exception { + htu.shutdownMiniCluster(); + } + + @Test + public void testMovingStagedFile() throws Exception { + Path stagingDirPath = + new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), STAGING_DIR)); + if (!dfs.exists(stagingDirPath)) { + dfs.mkdirs(stagingDirPath); + } + SecureBulkLoadManager.SecureBulkLoadListener listener = + new SecureBulkLoadManager.SecureBulkLoadListener(dfs, stagingDirPath.toString(), conf); + + //creating file to load + String srcFile = createHFileForFamilies(FAMILY); + Path srcPath = new Path(srcFile); + Assert.assertTrue(dfs.exists(srcPath)); + + Path stagedFamily = new Path(stagingDirPath, new Path(Bytes.toString(FAMILY))); + if (!dfs.exists(stagedFamily)) { + dfs.mkdirs(stagedFamily); + } + + //moving file to staging + String stagedFile = listener.prepareBulkLoad(FAMILY, srcFile, false, null); + Path stagedPath = new Path(stagedFile); + Assert.assertTrue(dfs.exists(stagedPath)); + Assert.assertFalse(dfs.exists(srcPath)); + + //moving files back to original location after a failed bulkload + listener.failedBulkLoad(FAMILY, stagedFile); + Assert.assertFalse(dfs.exists(stagedPath)); + Assert.assertTrue(dfs.exists(srcPath)); + } + + @Test + public void testMovingStagedFileWithCustomStageDir() throws Exception { + Path stagingDirPath = + new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), STAGING_DIR)); + if (!dfs.exists(stagingDirPath)) { + dfs.mkdirs(stagingDirPath); + } + SecureBulkLoadManager.SecureBulkLoadListener listener = + new SecureBulkLoadManager.SecureBulkLoadListener(dfs, stagingDirPath.toString(), conf); + + //creating file to load + String srcFile = createHFileForFamilies(FAMILY); + Path srcPath = new Path(srcFile); + Assert.assertTrue(dfs.exists(srcPath)); + + Path stagedFamily = new Path(stagingDirPath, new Path(Bytes.toString(FAMILY))); + if (!dfs.exists(stagedFamily)) { + dfs.mkdirs(stagedFamily); + } + + Path customStagingDirPath = + new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), CUSTOM_STAGING_DIR)); + Path customStagedFamily = new Path(customStagingDirPath, new Path(Bytes.toString(FAMILY))); + if (!dfs.exists(customStagedFamily)) { + dfs.mkdirs(customStagedFamily); + } + + //moving file to staging using a custom staging dir + String stagedFile = + listener.prepareBulkLoad(FAMILY, srcFile, false, customStagingDirPath.toString()); + Path stagedPath = new Path(stagedFile); + Assert.assertTrue(dfs.exists(stagedPath)); + Assert.assertFalse(dfs.exists(srcPath)); + + //moving files back to original location after a failed bulkload + listener.failedBulkLoad(FAMILY, stagedFile); + Assert.assertFalse(dfs.exists(stagedPath)); + Assert.assertTrue(dfs.exists(srcPath)); + } + + @Test + public void testCopiedStagedFile() throws Exception { + Path stagingDirPath = + new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), STAGING_DIR)); + if (!dfs.exists(stagingDirPath)) { + dfs.mkdirs(stagingDirPath); + } + SecureBulkLoadManager.SecureBulkLoadListener listener = + new SecureBulkLoadManager.SecureBulkLoadListener(dfs, stagingDirPath.toString(), conf); + + //creating file to load + String srcFile = createHFileForFamilies(FAMILY); + Path srcPath = new Path(srcFile); + Assert.assertTrue(dfs.exists(srcPath)); + + Path stagedFamily = new Path(stagingDirPath, new Path(Bytes.toString(FAMILY))); + if (!dfs.exists(stagedFamily)) { + dfs.mkdirs(stagedFamily); + } + + //copying file to staging + String stagedFile = listener.prepareBulkLoad(FAMILY, srcFile, true, null); + Path stagedPath = new Path(stagedFile); + Assert.assertTrue(dfs.exists(stagedPath)); + Assert.assertTrue(dfs.exists(srcPath)); + + //should do nothing because the original file was copied to staging + listener.failedBulkLoad(FAMILY, stagedFile); + Assert.assertTrue(dfs.exists(stagedPath)); + Assert.assertTrue(dfs.exists(srcPath)); + } + + @Test(expected = IOException.class) + public void testDeletedStagedFile() throws Exception { + Path stagingDirPath = + new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), STAGING_DIR)); + if (!dfs.exists(stagingDirPath)) { + dfs.mkdirs(stagingDirPath); + } + SecureBulkLoadManager.SecureBulkLoadListener listener = + new SecureBulkLoadManager.SecureBulkLoadListener(dfs, stagingDirPath.toString(), conf); + + //creating file to load + String srcFile = createHFileForFamilies(FAMILY); + Path srcPath = new Path(srcFile); + Assert.assertTrue(dfs.exists(srcPath)); + + Path stagedFamily = new Path(stagingDirPath, new Path(Bytes.toString(FAMILY))); + if (!dfs.exists(stagedFamily)) { + dfs.mkdirs(stagedFamily); + } + + //moving file to staging + String stagedFile = listener.prepareBulkLoad(FAMILY, srcFile, false, null); + Path stagedPath = new Path(stagedFile); + Assert.assertTrue(dfs.exists(stagedPath)); + Assert.assertFalse(dfs.exists(srcPath)); + + dfs.delete(stagedPath, false); + + //moving files back to original location after a failed bulkload + listener.failedBulkLoad(FAMILY, stagedFile); + } + + private String createHFileForFamilies(byte[] family) throws IOException { + HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf); + Path testDir = new Path(dfs.getWorkingDirectory() , new Path(name.getMethodName(), Bytes.toString(family))); + if(!dfs.exists(testDir)){ + dfs.mkdirs(testDir); + } + Path hfilePath = new Path(testDir, generateUniqueName(null)); + FSDataOutputStream out = dfs.createFile(hfilePath).build(); + try { + hFileFactory.withOutputStream(out); + hFileFactory.withFileContext(new HFileContextBuilder().build()); + HFile.Writer writer = hFileFactory.create(); + try { + writer.append(new KeyValue(ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY) + .setRow(randomBytes).setFamily(family).setQualifier(randomBytes).setTimestamp(0L) + .setType(KeyValue.Type.Put.getCode()).setValue(randomBytes).build())); + } finally { + writer.close(); + } + } finally { + out.close(); + } + return hfilePath.toString(); + } + + private static String generateUniqueName(final String suffix) { + String name = UUID.randomUUID().toString().replaceAll("-", ""); + if (suffix != null) name += suffix; + return name; + } + +} From 43cb6720a1f2623e77167e4613a1cb34625f482f Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Tue, 15 Mar 2022 20:13:46 +0800 Subject: [PATCH 29/32] HBASE-26611 Changing SFT implementation on disabled table is dangerous (#4082) Signed-off-by: Xiaolin Ha --- .../procedure/ModifyTableProcedure.java | 2 +- .../StoreFileTrackerValidationUtils.java | 16 +++++++++++++- .../TestChangeStoreFileTracker.java | 22 +++++++++++++++++++ 3 files changed, 38 insertions(+), 2 deletions(-) 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 b8e6b496995b..c5fa50c37427 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 @@ -321,7 +321,7 @@ private void prepareModify(final MasterProcedureEnv env) throws IOException { // check for store file tracker configurations StoreFileTrackerValidationUtils.checkForModifyTable(env.getMasterConfiguration(), - unmodifiedTableDescriptor, modifiedTableDescriptor); + unmodifiedTableDescriptor, modifiedTableDescriptor, !isTableEnabled(env)); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java index e6f6e854c890..38040bc4f006 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.regionserver.StoreUtils; @@ -94,7 +95,7 @@ public static void checkForCreateTable(Configuration conf, TableDescriptor table * {@code ModifyTableProcedure}. */ public static void checkForModifyTable(Configuration conf, TableDescriptor oldTable, - TableDescriptor newTable) throws IOException { + TableDescriptor newTable, boolean isTableDisabled) throws IOException { for (ColumnFamilyDescriptor newFamily : newTable.getColumnFamilies()) { ColumnFamilyDescriptor oldFamily = oldTable.getColumnFamily(newFamily.getName()); if (oldFamily == null) { @@ -133,6 +134,16 @@ public static void checkForModifyTable(Configuration conf, TableDescriptor oldTa newFamily.getNameAsString() + " of table " + newTable.getTableName()); } } else { + // do not allow changing from MIGRATION to its dst SFT implementation while the table is + // disabled. We need to open the HRegion to migrate the tracking information while the SFT + // implementation is MIGRATION, otherwise we may loss data. See HBASE-26611 for more + // details. + if (isTableDisabled) { + throw new TableNotEnabledException( + "Should not change store file tracker implementation from " + + StoreFileTrackerFactory.Trackers.MIGRATION.name() + " while table " + + newTable.getTableName() + " is disabled"); + } // we can only change to the dst tracker if (!newTracker.equals(oldDstTracker)) { throw new DoNotRetryIOException("Should migrate tracker to " + @@ -151,6 +162,9 @@ public static void checkForModifyTable(Configuration conf, TableDescriptor oldTa StoreFileTrackerFactory.getStoreFileTrackerName(oldTracker) + " for family " + newFamily.getNameAsString() + " of table " + newTable.getTableName()); } + // here we do not check whether the table is disabled, as after changing to MIGRATION, we + // still rely on the src SFT implementation to actually load the store files, so there + // will be no data loss problem. Class newSrcTracker = MigrationStoreFileTracker.getSrcTrackerClass(newConf); if (!oldTracker.equals(newSrcTracker)) { 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 index 110f896df8b8..4b54cd9131ba 100644 --- 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 @@ -19,6 +19,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; import java.io.IOException; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -26,6 +27,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; +import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -195,6 +197,26 @@ public void testModifyError8() throws IOException { UTIL.getAdmin().modifyTable(newTd); } + @Test + public void testModifyError9() throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build(); + UTIL.getAdmin().createTable(td); + UTIL.getAdmin().disableTable(td.getTableName()); + 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); + TableDescriptor newTd2 = TableDescriptorBuilder.newBuilder(td) + .setValue(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name()) + .build(); + // changing from MIGRATION while table is disabled is not allowed + assertThrows(TableNotEnabledException.class, () -> UTIL.getAdmin().modifyTable(newTd2)); + } + private String getStoreFileName(TableName table, byte[] family) { return Iterables .getOnlyElement(Iterables.getOnlyElement(UTIL.getMiniHBaseCluster().getRegions(table)) From 5acfe6dcd38af46a7d5a04dff0889e1be7c5461a Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 16 Mar 2022 19:34:30 +0800 Subject: [PATCH 30/32] HBASE-26837 Set SFT config when creating TableDescriptor in TestCloneSnapshotProcedure (#4226) Signed-off-by: Xiaolin Ha (cherry picked from commit 5b28d7665219c24863eebeaf9aa8dcd6c26693bc) --- .../master/procedure/TestCloneSnapshotProcedure.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java index 409e912cb6c4..bbe3f8ec7237 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java @@ -33,6 +33,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.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -95,8 +96,11 @@ private int getNumReplicas() { return 1; } - public static TableDescriptor createTableDescriptor(TableName tableName, byte[]... family) { - TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); + private static TableDescriptor createTableDescriptor(TableName tableName, byte[]... family) { + TableDescriptorBuilder builder = + TableDescriptorBuilder.newBuilder(tableName).setValue(StoreFileTrackerFactory.TRACKER_IMPL, + UTIL.getConfiguration().get(StoreFileTrackerFactory.TRACKER_IMPL, + StoreFileTrackerFactory.Trackers.DEFAULT.name())); Stream.of(family).map(ColumnFamilyDescriptorBuilder::of) .forEachOrdered(builder::setColumnFamily); return builder.build(); From 541d748e841022ce53b7f39ca4ef8c81afcf8028 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Fri, 25 Mar 2022 10:04:13 +0000 Subject: [PATCH 31/32] HBASE-26881 Backport HBASE-25368 to branch-2 (#4267) Signed-off-by: Andrew Purtell --- .../hbase/client/RawAsyncHBaseAdmin.java | 86 ++++++++++--------- .../hadoop/hbase/client/RegionInfo.java | 20 ++++- .../hadoop/hbase/client/TestAdmin1.java | 19 ++++ .../hadoop/hbase/client/TestAdmin2.java | 9 +- 4 files changed, 86 insertions(+), 48 deletions(-) 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 3ae948bb8d50..29698c201336 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 @@ -2394,51 +2394,55 @@ CompletableFuture getRegionLocation(byte[] regionNameOrEncodedR if (regionNameOrEncodedRegionName == null) { return failedFuture(new IllegalArgumentException("Passed region name can't be null")); } - try { - CompletableFuture> future; - if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) { - String encodedName = Bytes.toString(regionNameOrEncodedRegionName); - if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) { - // old format encodedName, should be meta region - future = connection.registry.getMetaRegionLocations() - .thenApply(locs -> Stream.of(locs.getRegionLocations()) - .filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst()); - } else { - future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(metaTable, - regionNameOrEncodedRegionName); - } + + CompletableFuture> future; + if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) { + String encodedName = Bytes.toString(regionNameOrEncodedRegionName); + if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) { + // old format encodedName, should be meta region + future = connection.registry.getMetaRegionLocations() + .thenApply(locs -> Stream.of(locs.getRegionLocations()) + .filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst()); } else { - RegionInfo regionInfo = - MetaTableAccessor.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName); - if (regionInfo.isMetaRegion()) { - future = connection.registry.getMetaRegionLocations() - .thenApply(locs -> Stream.of(locs.getRegionLocations()) - .filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId()) - .findFirst()); - } else { - future = - AsyncMetaTableAccessor.getRegionLocation(metaTable, regionNameOrEncodedRegionName); - } + future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(metaTable, + regionNameOrEncodedRegionName); + } + } else { + // Not all regionNameOrEncodedRegionName here is going to be a valid region name, + // it needs to throw out IllegalArgumentException in case tableName is passed in. + RegionInfo regionInfo; + try { + regionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName); + } catch (IOException ioe) { + return failedFuture(new IllegalArgumentException(ioe.getMessage())); } - CompletableFuture returnedFuture = new CompletableFuture<>(); - addListener(future, (location, err) -> { - if (err != null) { - returnedFuture.completeExceptionally(err); - return; - } - if (!location.isPresent() || location.get().getRegion() == null) { - returnedFuture.completeExceptionally( - new UnknownRegionException("Invalid region name or encoded region name: " + - Bytes.toStringBinary(regionNameOrEncodedRegionName))); - } else { - returnedFuture.complete(location.get()); - } - }); - return returnedFuture; - } catch (IOException e) { - return failedFuture(e); + if (regionInfo.isMetaRegion()) { + future = connection.registry.getMetaRegionLocations() + .thenApply(locs -> Stream.of(locs.getRegionLocations()) + .filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId()) + .findFirst()); + } else { + future = + AsyncMetaTableAccessor.getRegionLocation(metaTable, regionNameOrEncodedRegionName); + } } + + CompletableFuture returnedFuture = new CompletableFuture<>(); + addListener(future, (location, err) -> { + if (err != null) { + returnedFuture.completeExceptionally(err); + return; + } + if (!location.isPresent() || location.get().getRegion() == null) { + returnedFuture.completeExceptionally( + new UnknownRegionException("Invalid region name or encoded region name: " + + Bytes.toStringBinary(regionNameOrEncodedRegionName))); + } else { + returnedFuture.complete(location.get()); + } + }); + return returnedFuture; } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java index e105dbc3ad81..81dc63346c4a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java @@ -362,9 +362,25 @@ static byte[] getStartKey(final byte[] regionName) throws IOException { * @return True if regionName represents an encoded name. */ @InterfaceAudience.Private // For use by internals only. - public static boolean isEncodedRegionName(byte[] regionName) { + static boolean isEncodedRegionName(byte[] regionName) { // If not parseable as region name, presume encoded. TODO: add stringency; e.g. if hex. - return parseRegionNameOrReturnNull(regionName) == null && regionName.length <= MD5_HEX_LENGTH; + if (parseRegionNameOrReturnNull(regionName) == null) { + if (regionName.length > MD5_HEX_LENGTH) { + return false; + } else if (regionName.length == MD5_HEX_LENGTH) { + return true; + } else { + String encodedName = Bytes.toString(regionName); + try { + Integer.parseInt(encodedName); + // If this is a valid integer, it could be hbase:meta's encoded region name. + return true; + } catch(NumberFormatException er) { + return false; + } + } + } + return false; } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index fb3b4dd8bfd0..d85feda69a5b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -130,6 +131,24 @@ public void testSplitFlushCompactUnknownTable() throws InterruptedException { assertTrue(exception instanceof TableNotFoundException); } + @Test + public void testCompactATableWithSuperLongTableName() throws Exception { + TableName tableName = TableName.valueOf(name.getMethodName()); + TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of("fam1")).build(); + try { + ADMIN.createTable(htd); + assertThrows(IllegalArgumentException.class, + () -> ADMIN.majorCompactRegion(tableName.getName())); + + assertThrows(IllegalArgumentException.class, + () -> ADMIN.majorCompactRegion(Bytes.toBytes("abcd"))); + } finally { + ADMIN.disableTable(tableName); + ADMIN.deleteTable(tableName); + } + } + @Test public void testCompactionTimestamps() throws Exception { TableName tableName = TableName.valueOf(name.getMethodName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 5c855f4b47f7..6c16416eac7e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -316,11 +317,9 @@ public void testCloseRegionIfInvalidRegionNameIsPassed() throws Exception { if (!regionInfo.isMetaRegion()) { if (regionInfo.getRegionNameAsString().contains(name)) { info = regionInfo; - try { - ADMIN.unassign(Bytes.toBytes("sample"), true); - } catch (UnknownRegionException nsre) { - // expected, ignore it - } + assertThrows(UnknownRegionException.class, + () -> ADMIN.unassign(Bytes.toBytes( + "test,,1358563771069.acc1ad1b7962564fc3a43e5907e8db33."), true)); } } } From 4e80f0ea79ac7eb7cdc23297798290545ca73872 Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Fri, 18 Mar 2022 13:38:51 -0700 Subject: [PATCH 32/32] HBASE-26826 Backport StoreFileTracker (HBASE-26067, HBASE-26584, and others) to branch-2.5 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previous cherry picks: commit 6aaef89 HBASE-26064 Introduce a StoreFileTracker to abstract the store file tracking logic commit 43b40e9 HBASE-25988 Store the store file list by a file #3578) commit 6e05376 HBASE-26079 Use StoreFileTracker when splitting and merging #3617) commit 090b2fe HBASE-26224 HBASE-26224 Introduce a MigrationStoreFileTracker to support migratin… #3656) commit 0ee1689 HBASE-26246 Persist the StoreFileTracker configurations to TableDescriptor when creating table #3666) commit 2052e80 HBASE-26248 Should find a suitable way to let users specify the store… #3665) commit 5ff0f98 HBASE-26264 Add more checks to prevent misconfiguration on store file… #3681) commit fc4f6d1 HBASE-26280 HBASE-26280 Use store file tracker when snapshoting #3685) commit 06db852 HBASE-26326 CreateTableProcedure fails when FileBasedStoreFileTracker… #3721) commit e4e7cf8 HBASE-26386 Refactor StoreFileTracker implementations to expose the s… #3774) commit 08d1171 HBASE-26328 Clone snapshot doesn't load reference files into FILE SFT impl #3749) commit 8bec26e HBASE-26263 [Rolling Upgrading] Persist the StoreFileTracker configur… #3700) commit a288365 HBASE-26271: Cleanup the broken store files under data directory #3786) commit d00b5fa HBASE-26454 CreateTableProcedure still relies on temp dir and renames… #3845) commit 771e552 HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot commit f16b7b1 HBASE-26265 Update ref guide to mention the new store file tracker im… #3942) commit 755b3b4 HBASE-26585 Add SFT configuration to META table descriptor when creating META #3998) commit 39c42c7 HBASE-26639 The implementation of TestMergesSplitsAddToTracker is pro… #4010) commit 6e1f5b7 HBASE-26586 Should not rely on the global config when setting SFT implementation for a table while upgrading #4006) commit f1dd865 HBASE-26654 ModifyTableDescriptorProcedure shoud load TableDescriptor… #4034) commit 8fbc9a2 HBASE-26674 Should modify filesCompacting under storeWriteLock #4040) commit 5aa0fd2 HBASE-26675 Data race on Compactor.writer #4035) commit 3021c58 HBASE-26700 The way we bypass broken track file is not enough in Stor… #4055) commit a8b68c9 HBASE-26690 Modify FSTableDescriptors to not rely on renaming when wr… #4054) commit dffeb8e HBASE-26587 Introduce a new Admin API to change SFT implementation (#… #4080) commit b265fe5 HBASE-26673 Implement a shell command for change SFT implementation #4113) commit 4cdb380 HBASE-26640 Reimplement master local region initialization to better … #4111) commit 77bb153 HBASE-26707: Reduce number of renames during bulkload (#4066) #4122) commit a4b192e HBASE-26611 Changing SFT implementation on disabled table is dangerous #4082) commit d3629bb HBASE-26837 Set SFT config when creating TableDescriptor in TestClone… #4226) commit 541d748 HBASE-26881 Backport HBASE-25368 to branch-2 (#4267) Fixups for precommit error prone, checkstyle, and javadoc warnings after applying cherry picks. Signed-off-by: Josh Elser Reviewed-by: Wellington Ramos Chevreuil --- .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 8 ++++---- .../hadoop/hbase/client/TableDescriptorBuilder.java | 5 ++--- .../IntegrationTestFileBasedSFTBulkLoad.java | 3 ++- .../hadoop/hbase/regionserver/AbstractMemStore.java | 3 +-- .../hbase/regionserver/CompactingMemStore.java | 2 +- .../hadoop/hbase/regionserver/HRegionServer.java | 3 ++- .../hbase/regionserver/SecureBulkLoadManager.java | 2 +- .../hbase/snapshot/RestoreSnapshotHelper.java | 2 -- .../hadoop/hbase/regionserver/TestBulkloadBase.java | 11 +++++++---- .../regionserver/TestMergesSplitsAddToTracker.java | 1 - .../regionserver/TestSecureBulkloadListener.java | 13 ++++++++----- .../hbase/regionserver/TestStripeStoreEngine.java | 1 - .../compactions/TestDateTieredCompactor.java | 1 - .../compactions/TestStripeCompactor.java | 1 - .../hbase/snapshot/MobSnapshotTestingUtils.java | 11 ++++++----- 15 files changed, 34 insertions(+), 33 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index b87874e1994a..6cf018468ebb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -2868,10 +2868,10 @@ private Future internalRestoreSnapshotAsync(final String snapshotName, @Override protected RestoreSnapshotResponse rpcCall() throws Exception { final RestoreSnapshotRequest.Builder builder = RestoreSnapshotRequest.newBuilder() - .setSnapshot(snapshot) - .setNonceGroup(nonceGroup) - .setNonce(nonce) - .setRestoreACL(restoreAcl); + .setSnapshot(snapshot) + .setNonceGroup(nonceGroup) + .setNonce(nonce) + .setRestoreACL(restoreAcl); if (customSFT != null) { builder.setCustomSFT(customSFT); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index d9a15154a313..c84388dcbbf0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -818,9 +818,8 @@ public ModifyableTableDescriptor removeValue(Bytes key) { /** * Remove metadata represented by the key from the {@link #values} map * - * @param key Key whose key and value we're to remove from TableDescriptor - * parameters. - * @return the modifyable TD + * @param key Key whose key and value we're to remove from TableDescriptor parameters + * @return the modifiable TD */ public ModifyableTableDescriptor removeValue(final byte[] key) { return removeValue(new Bytes(key)); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestFileBasedSFTBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestFileBasedSFTBulkLoad.java index 93e51fff8aab..ca36ff8effa4 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestFileBasedSFTBulkLoad.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestFileBasedSFTBulkLoad.java @@ -66,7 +66,8 @@ @Category(IntegrationTests.class) public class IntegrationTestFileBasedSFTBulkLoad extends IntegrationTestBulkLoad { - private static final Logger LOG = LoggerFactory.getLogger(IntegrationTestFileBasedSFTBulkLoad.class); + private static final Logger LOG = + LoggerFactory.getLogger(IntegrationTestFileBasedSFTBulkLoad.class); private static String NUM_MAPS_KEY = "hbase.IntegrationTestBulkLoad.numMaps"; private static String NUM_IMPORT_ROUNDS_KEY = "hbase.IntegrationTestBulkLoad.numImportRounds"; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java index 56dab21baf2e..0cef4a91fb7d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java @@ -232,8 +232,7 @@ public long timeOfOldestEdit() { } /** - * This method is protected under {@link HStore#lock} write lock,
- * and this method is used by {@link HStore#updateStorefiles} after flushing is completed.
+ * This method is protected under HStore write lock.
* The passed snapshot was successfully persisted; it can be let go. * @param id Id of the snapshot to clean out. * @see MemStore#snapshot() diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java index eac5fc110c57..cd3caa9c3ebd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java @@ -395,7 +395,7 @@ public String getFamilyName() { } /** - * This method is protected under {@link HStore#lock} read lock. + * This method is protected under HStore read lock. */ @Override public List getScanners(long readPt) throws IOException { 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 2bef9388021f..990b4da46f6b 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 @@ -2262,7 +2262,8 @@ private void initializeThreads() { double brokenStoreFileCleanerDelayJitter = conf.getDouble( BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY_JITTER, BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER); - double jitterRate = (ThreadLocalRandom.current().nextDouble() - 0.5D) * brokenStoreFileCleanerDelayJitter; + double jitterRate = (ThreadLocalRandom.current().nextDouble() - 0.5D) * + brokenStoreFileCleanerDelayJitter; long jitterValue = Math.round(brokenStoreFileCleanerDelay * jitterRate); this.brokenStoreFileCleaner = new BrokenStoreFileCleaner((int) (brokenStoreFileCleanerDelay + jitterValue), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java index dbc5f72e37b6..ee028f1487a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java @@ -364,7 +364,7 @@ public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration co @Override public String prepareBulkLoad(final byte[] family, final String srcPath, boolean copyFile, - String customStaging ) throws IOException { + String customStaging) throws IOException { Path p = new Path(srcPath); //store customStaging for failedBulkLoad 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 17406445fc3a..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 @@ -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; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java index 86e41ede8294..fbf71e29b3e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java @@ -59,10 +59,10 @@ import org.junit.Rule; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; @RunWith(Parameterized.class) public class TestBulkloadBase { @@ -135,7 +135,8 @@ protected HRegion testRegionWithFamiliesAndSpecifiedTableName(TableName tableNam } protected HRegion testRegionWithFamilies(byte[]... families) throws IOException { - TableName tableName = TableName.valueOf(name.getMethodName().substring(0, name.getMethodName().indexOf("["))); + TableName tableName = + TableName.valueOf(name.getMethodName().substring(0, name.getMethodName().indexOf("["))); return testRegionWithFamiliesAndSpecifiedTableName(tableName, families); } @@ -175,7 +176,9 @@ private String createHFileForFamilies(byte[] family) throws IOException { private static String generateUniqueName(final String suffix) { String name = UUID.randomUUID().toString().replaceAll("-", ""); - if (suffix != null) name += suffix; + if (suffix != null) { + name += suffix; + } return name; } 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 703d6193e5e5..b0520e353d1b 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 @@ -234,7 +234,6 @@ private void validateDaughterRegionsFiles(HRegion region, String orignalFileName 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(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkloadListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkloadListener.java index bf3732a821da..7a6e411dfab0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkloadListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkloadListener.java @@ -17,6 +17,9 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.io.IOException; +import java.util.Random; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -42,9 +45,6 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; -import java.io.IOException; -import java.util.Random; -import java.util.UUID; /** * Tests for failedBulkLoad logic to make sure staged files are returned to their original location @@ -232,7 +232,8 @@ public void testDeletedStagedFile() throws Exception { private String createHFileForFamilies(byte[] family) throws IOException { HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf); - Path testDir = new Path(dfs.getWorkingDirectory() , new Path(name.getMethodName(), Bytes.toString(family))); + Path testDir = new Path(dfs.getWorkingDirectory(), + new Path(name.getMethodName(), Bytes.toString(family))); if(!dfs.exists(testDir)){ dfs.mkdirs(testDir); } @@ -257,7 +258,9 @@ private String createHFileForFamilies(byte[] family) throws IOException { private static String generateUniqueName(final String suffix) { String name = UUID.randomUUID().toString().replaceAll("-", ""); - if (suffix != null) name += suffix; + if (suffix != null) { + name += suffix; + } return name; } 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 e30c9d9f5015..24cf3e96ce75 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 @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; 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 dd844080aa82..dfbda26147de 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 @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; 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 0ae584c5aa9b..e834d66112c4 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 @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; 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 4fb1ae2e1f04..cb0fad6c8fe2 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 @@ -59,8 +59,9 @@ public static void createMobTable(final HBaseTestingUtility util, final TableNam storeFileTracker, families); } - public static void createPreSplitMobTable(final HBaseTestingUtility util, final TableName tableName, - int nRegions, final byte[]... families) throws IOException, InterruptedException { + public static void createPreSplitMobTable(final HBaseTestingUtility util, + final TableName tableName, int nRegions, final byte[]... families) + throws IOException, InterruptedException { createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(nRegions), 1, families); } @@ -77,9 +78,9 @@ public static void createMobTable(final HBaseTestingUtility util, final TableNam createMobTable(util, tableName, splitKeys, regionReplication, storeFileTracker, null, families); } - public static void createMobTable(HBaseTestingUtility util, TableName tableName, byte[][] splitKeys, - int regionReplication, String storeFileTracker, String cpClassName, byte[]... families) - throws IOException, InterruptedException { + public static void createMobTable(HBaseTestingUtility 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) {