-
Notifications
You must be signed in to change notification settings - Fork 3.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
HBASE-25991 Do compaction on compaction server (#3425)
Signed-off-by: Duo Zhang <zhangduo@apache.org> Signed-off-by: Michael Stack <stack@apache.org>
- Loading branch information
1 parent
da0fa30
commit f19f92d
Showing
22 changed files
with
1,475 additions
and
234 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
182 changes: 182 additions & 0 deletions
182
...e-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionFilesCache.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,182 @@ | ||
/** | ||
* 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.compactionserver; | ||
|
||
import java.util.HashSet; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.ConcurrentMap; | ||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; | ||
import org.apache.hadoop.hbase.client.RegionInfo; | ||
import org.apache.yetus.audience.InterfaceAudience; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; | ||
|
||
/** | ||
* Since we do not maintain StoreFileManager in compaction server(can't refresh when flush). we use | ||
* external storage(this class) to record compacting and compacted files. This storage is in memory | ||
* and only used by CompactionServer,(RegionServer not use it).This storage never do hfile movement | ||
* or deletion. RS does file movement still. | ||
*/ | ||
@InterfaceAudience.Private | ||
class CompactionFilesCache { | ||
private static Logger LOG = LoggerFactory.getLogger(CompactionFilesCache.class); | ||
private final ConcurrentMap<String, ConcurrentMap<String, Set<String>>> selectedFiles = | ||
new ConcurrentHashMap<>(); | ||
private final ConcurrentMap<String, ConcurrentMap<String, Set<String>>> compactedFiles = | ||
new ConcurrentHashMap<>(); | ||
/** | ||
* Mark files as completed, called after CS finished compaction and RS accepted the results of | ||
* this compaction, these compacted files will be deleted by RS if no reader referenced to them. | ||
*/ | ||
boolean addCompactedFiles(RegionInfo regionInfo, ColumnFamilyDescriptor cfd, | ||
List<String> compactedFiles) { | ||
Set<String> compactedFileSet = getCompactedStoreFilesInternal(regionInfo, cfd); | ||
synchronized (compactedFileSet) { | ||
compactedFileSet.addAll(compactedFiles); | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("Mark files as compacted, region: {}, cf, files: {}", regionInfo, | ||
cfd.getNameAsString(), compactedFileSet); | ||
} | ||
} | ||
return true; | ||
} | ||
|
||
/** | ||
* Mark files as selected, called after the files are selected and before the compaction is | ||
* started. Avoid a file is selected twice in two compaction. | ||
* @return True if these files don't be selected, false if these files are already selected. | ||
*/ | ||
boolean addSelectedFiles(RegionInfo regionInfo, ColumnFamilyDescriptor cfd, | ||
List<String> selectedFiles) { | ||
Set<String> selectedFileSet = getSelectedStoreFilesInternal(regionInfo, cfd); | ||
synchronized (selectedFileSet) { | ||
for (String selectedFile : selectedFiles) { | ||
if (selectedFileSet.contains(selectedFile)) { | ||
return false; | ||
} | ||
} | ||
selectedFileSet.addAll(selectedFiles); | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("Mark files are selected, region: {}, cf: {}, files: {}", | ||
regionInfo.getEncodedName(), cfd.getNameAsString(), selectedFiles); | ||
} | ||
} | ||
return true; | ||
} | ||
|
||
/** | ||
* Get files which are compacted, called before select files to do compaction. Thread-safe | ||
* @return The files which are compacted | ||
*/ | ||
Set<String> getCompactedStoreFiles(RegionInfo regionInfo, ColumnFamilyDescriptor cfd) { | ||
Set<String> compactedFiles = getStoreFiles(this.compactedFiles, regionInfo, cfd); | ||
synchronized (compactedFiles) { | ||
return ImmutableSet.copyOf(compactedFiles); | ||
} | ||
} | ||
|
||
/** | ||
* Get files which are compacting, called before select files to do compaction. Thread-safe | ||
* @return The files which are compacting | ||
*/ | ||
Set<String> getSelectedStoreFiles(RegionInfo regionInfo, ColumnFamilyDescriptor cfd) { | ||
Set<String> selectedFiles = getStoreFiles(this.selectedFiles, regionInfo, cfd); | ||
synchronized (selectedFiles) { | ||
return ImmutableSet.copyOf(selectedFiles); | ||
} | ||
} | ||
|
||
/** | ||
* Get files which are compacted, we use this return object as lock | ||
* @return The files which are compacted as lock object | ||
*/ | ||
Object getCompactedStoreFilesAsLock(RegionInfo regionInfo, ColumnFamilyDescriptor cfd) { | ||
return getCompactedStoreFilesInternal(regionInfo, cfd); | ||
} | ||
|
||
/** | ||
* Get files which are compacting, we use this return object as lock | ||
* @return The files which are compacting as lock object | ||
*/ | ||
Object getSelectedStoreFilesAsLock(RegionInfo regionInfo, ColumnFamilyDescriptor cfd) { | ||
return getSelectedStoreFilesInternal(regionInfo, cfd); | ||
} | ||
|
||
/** | ||
* Get files which are compacted, called before select files to do compaction. No-thread-safe | ||
* @return The files which are compacted | ||
*/ | ||
private Set<String> getCompactedStoreFilesInternal(RegionInfo regionInfo, | ||
ColumnFamilyDescriptor cfd) { | ||
return getStoreFiles(this.compactedFiles, regionInfo, cfd); | ||
} | ||
|
||
/** | ||
* Get files which are compacting, called before select files to do compaction. No-thread-safe | ||
* @return The files which are compacting | ||
*/ | ||
private Set<String> getSelectedStoreFilesInternal(RegionInfo regionInfo, | ||
ColumnFamilyDescriptor cfd) { | ||
return getStoreFiles(this.selectedFiles, regionInfo, cfd); | ||
} | ||
|
||
private Set<String> getStoreFiles( | ||
ConcurrentMap<String, ConcurrentMap<String, Set<String>>> fileMap, RegionInfo regionInfo, | ||
ColumnFamilyDescriptor cfd) { | ||
String encodedName = regionInfo.getEncodedName(); | ||
String family = cfd.getNameAsString(); | ||
Map<String, Set<String>> familyFilesMap = | ||
fileMap.computeIfAbsent(encodedName, v -> new ConcurrentHashMap<>()); | ||
return familyFilesMap.computeIfAbsent(family, v -> new HashSet<>()); | ||
} | ||
|
||
/** | ||
* Remove files from selected, called: | ||
* 1. after the compaction is failed; | ||
* 2. after the compaction is finished and report to RS failed; | ||
* 3. after the compaction is finished and report to RS succeeded (and will mark these files | ||
* as compacted). | ||
*/ | ||
void removeSelectedFiles(RegionInfo regionInfo, ColumnFamilyDescriptor cfd, | ||
List<String> selectedFiles) { | ||
Set<String> selectedFileSet = getSelectedStoreFilesInternal(regionInfo, cfd); | ||
synchronized (selectedFileSet) { | ||
selectedFileSet.removeAll(selectedFiles); | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("Remove files from selected, region: {}, cf: {}, files: {}", | ||
regionInfo.getEncodedName(), cfd.getNameAsString(), selectedFiles); | ||
} | ||
} | ||
} | ||
|
||
/** | ||
* Remove compacted files which are already deleted by RS | ||
*/ | ||
void cleanupCompactedFiles(RegionInfo regionInfo, ColumnFamilyDescriptor cfd, | ||
Set<String> storeFileNames) { | ||
Set<String> compactedFileSet = getCompactedStoreFilesInternal(regionInfo, cfd); | ||
synchronized (compactedFileSet) { | ||
compactedFileSet.retainAll(storeFileNames); | ||
} | ||
} | ||
} |
Oops, something went wrong.