HBASE-22202 Fix new findbugs issues after we upgrade hbase-thirdparty dependencies
This commit is contained in:
parent
94d9dc1e84
commit
f77bde3722
|
@ -21,7 +21,6 @@ import java.io.FileNotFoundException;
|
|||
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;
|
||||
|
@ -31,7 +30,9 @@ import java.util.concurrent.ThreadFactory;
|
|||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -49,10 +50,8 @@ import org.apache.hadoop.io.MultipleIOException;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Function;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Collections2;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Utility class to handle the removal of HFiles (or the respective {@link HStoreFile StoreFiles})
|
||||
|
@ -158,15 +157,15 @@ public class HFileArchiver {
|
|||
}
|
||||
|
||||
// convert the files in the region to a File
|
||||
toArchive.addAll(Lists.transform(Arrays.asList(storeDirs), getAsFile));
|
||||
Stream.of(storeDirs).map(getAsFile).forEachOrdered(toArchive::add);
|
||||
LOG.debug("Archiving " + toArchive);
|
||||
List<File> failedArchive = resolveAndArchive(fs, regionArchiveDir, toArchive,
|
||||
EnvironmentEdgeManager.currentTime());
|
||||
if (!failedArchive.isEmpty()) {
|
||||
throw new FailedArchiveException("Failed to archive/delete all the files for region:"
|
||||
+ regionDir.getName() + " into " + regionArchiveDir
|
||||
+ ". Something is probably awry on the filesystem.",
|
||||
Collections2.transform(failedArchive, FUNC_FILE_TO_PATH));
|
||||
throw new FailedArchiveException(
|
||||
"Failed to archive/delete all the files for region:" + regionDir.getName() + " into " +
|
||||
regionArchiveDir + ". Something is probably awry on the filesystem.",
|
||||
failedArchive.stream().map(FUNC_FILE_TO_PATH).collect(Collectors.toList()));
|
||||
}
|
||||
// if that was successful, then we delete the region
|
||||
return deleteRegionWithoutArchiving(fs, regionDir);
|
||||
|
@ -269,7 +268,7 @@ public class HFileArchiver {
|
|||
}
|
||||
|
||||
FileStatusConverter getAsFile = new FileStatusConverter(fs);
|
||||
Collection<File> toArchive = Lists.transform(Arrays.asList(storeFiles), getAsFile);
|
||||
Collection<File> toArchive = Stream.of(storeFiles).map(getAsFile).collect(Collectors.toList());
|
||||
Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, parent, family);
|
||||
|
||||
// do the actual archive
|
||||
|
@ -279,7 +278,7 @@ public class HFileArchiver {
|
|||
throw new FailedArchiveException("Failed to archive/delete all the files for region:"
|
||||
+ Bytes.toString(parent.getRegionName()) + ", family:" + Bytes.toString(family)
|
||||
+ " into " + storeArchiveDir + ". Something is probably awry on the filesystem.",
|
||||
Collections2.transform(failedArchive, FUNC_FILE_TO_PATH));
|
||||
failedArchive.stream().map(FUNC_FILE_TO_PATH).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -328,17 +327,18 @@ public class HFileArchiver {
|
|||
|
||||
// Wrap the storefile into a File
|
||||
StoreToFile getStorePath = new StoreToFile(fs);
|
||||
Collection<File> storeFiles = Collections2.transform(compactedFiles, getStorePath);
|
||||
Collection<File> storeFiles =
|
||||
compactedFiles.stream().map(getStorePath).collect(Collectors.toList());
|
||||
|
||||
// do the actual archive
|
||||
List<File> failedArchive = resolveAndArchive(fs, storeArchiveDir, storeFiles,
|
||||
EnvironmentEdgeManager.currentTime());
|
||||
List<File> failedArchive =
|
||||
resolveAndArchive(fs, storeArchiveDir, storeFiles, EnvironmentEdgeManager.currentTime());
|
||||
|
||||
if (!failedArchive.isEmpty()){
|
||||
throw new FailedArchiveException("Failed to archive/delete all the files for region:"
|
||||
+ Bytes.toString(regionInfo.getRegionName()) + ", family:" + Bytes.toString(family)
|
||||
+ " into " + storeArchiveDir + ". Something is probably awry on the filesystem.",
|
||||
Collections2.transform(failedArchive, FUNC_FILE_TO_PATH));
|
||||
failedArchive.stream().map(FUNC_FILE_TO_PATH).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -698,8 +698,10 @@ public class HFileArchiver {
|
|||
|
||||
@Override
|
||||
public Collection<File> getChildren() throws IOException {
|
||||
if (fs.isFile(file)) return Collections.emptyList();
|
||||
return Collections2.transform(Arrays.asList(fs.listStatus(file)), getAsFile);
|
||||
if (fs.isFile(file)) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return Stream.of(fs.listStatus(file)).map(getAsFile).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,14 +23,12 @@ import java.util.Objects;
|
|||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
||||
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
|
||||
|
@ -101,12 +99,9 @@ public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
|
|||
public Iterable<Entry<RegionInfo, Long>> filterBySubject(String namespace) {
|
||||
rlock.lock();
|
||||
try {
|
||||
return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<RegionInfo,Long>>() {
|
||||
@Override
|
||||
public boolean apply(Entry<RegionInfo,Long> input) {
|
||||
return namespace.equals(input.getKey().getTable().getNamespaceAsString());
|
||||
}
|
||||
});
|
||||
return regionUsage.entrySet().stream()
|
||||
.filter(entry -> namespace.equals(entry.getKey().getTable().getNamespaceAsString()))
|
||||
.collect(Collectors.toList());
|
||||
} finally {
|
||||
rlock.unlock();
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.Objects;
|
|||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
||||
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -37,9 +37,9 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
|
||||
|
@ -144,15 +144,11 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Entry<RegionInfo,Long>> filterBySubject(TableName table) {
|
||||
public Iterable<Entry<RegionInfo, Long>> filterBySubject(TableName table) {
|
||||
rlock.lock();
|
||||
try {
|
||||
return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<RegionInfo,Long>>() {
|
||||
@Override
|
||||
public boolean apply(Entry<RegionInfo,Long> input) {
|
||||
return table.equals(input.getKey().getTable());
|
||||
}
|
||||
});
|
||||
return regionUsage.entrySet().stream()
|
||||
.filter(entry -> table.equals(entry.getKey().getTable())).collect(Collectors.toList());
|
||||
} finally {
|
||||
rlock.unlock();
|
||||
}
|
||||
|
|
|
@ -17,12 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Function;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Ordering;
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
import java.util.function.Function;
|
||||
import java.util.function.ToLongFunction;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
|
@ -37,32 +34,23 @@ final class StoreFileComparators {
|
|||
* ordering, then bulkLoadTime. If there are ties, the path name is used as a tie-breaker.
|
||||
*/
|
||||
public static final Comparator<HStoreFile> SEQ_ID =
|
||||
Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
|
||||
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
|
||||
Ordering.natural().onResultOf(new GetBulkTime()),
|
||||
Ordering.natural().onResultOf(new GetPathName())));
|
||||
Comparator.comparingLong(HStoreFile::getMaxSequenceId)
|
||||
.thenComparing(Comparator.comparingLong(new GetFileSize()).reversed())
|
||||
.thenComparingLong(new GetBulkTime()).thenComparing(new GetPathName());
|
||||
|
||||
/**
|
||||
* Comparator for time-aware compaction. SeqId is still the first ordering criterion to maintain
|
||||
* MVCC.
|
||||
*/
|
||||
public static final Comparator<HStoreFile> SEQ_ID_MAX_TIMESTAMP =
|
||||
Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
|
||||
Ordering.natural().onResultOf(new GetMaxTimestamp()),
|
||||
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
|
||||
Ordering.natural().onResultOf(new GetBulkTime()),
|
||||
Ordering.natural().onResultOf(new GetPathName())));
|
||||
Comparator.comparingLong(HStoreFile::getMaxSequenceId).thenComparingLong(new GetMaxTimestamp())
|
||||
.thenComparing(Comparator.comparingLong(new GetFileSize()).reversed())
|
||||
.thenComparingLong(new GetBulkTime()).thenComparing(new GetPathName());
|
||||
|
||||
private static class GetSeqId implements Function<HStoreFile, Long> {
|
||||
@Override
|
||||
public Long apply(HStoreFile sf) {
|
||||
return sf.getMaxSequenceId();
|
||||
}
|
||||
}
|
||||
private static class GetFileSize implements ToLongFunction<HStoreFile> {
|
||||
|
||||
private static class GetFileSize implements Function<HStoreFile, Long> {
|
||||
@Override
|
||||
public Long apply(HStoreFile sf) {
|
||||
public long applyAsLong(HStoreFile sf) {
|
||||
if (sf.getReader() != null) {
|
||||
return sf.getReader().length();
|
||||
} else {
|
||||
|
@ -73,23 +61,26 @@ final class StoreFileComparators {
|
|||
}
|
||||
}
|
||||
|
||||
private static class GetBulkTime implements Function<HStoreFile, Long> {
|
||||
private static class GetBulkTime implements ToLongFunction<HStoreFile> {
|
||||
|
||||
@Override
|
||||
public Long apply(HStoreFile sf) {
|
||||
public long applyAsLong(HStoreFile sf) {
|
||||
return sf.getBulkLoadTimestamp().orElse(Long.MAX_VALUE);
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetPathName implements Function<HStoreFile, String> {
|
||||
|
||||
@Override
|
||||
public String apply(HStoreFile sf) {
|
||||
return sf.getPath().getName();
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetMaxTimestamp implements Function<HStoreFile, Long> {
|
||||
private static class GetMaxTimestamp implements ToLongFunction<HStoreFile> {
|
||||
|
||||
@Override
|
||||
public Long apply(HStoreFile sf) {
|
||||
public long applyAsLong(HStoreFile sf) {
|
||||
return sf.getMaximumTimestamp().orElse(Long.MAX_VALUE);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,7 +117,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
|
|||
candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck)
|
||||
throws IOException {
|
||||
if (!tryingMajor) {
|
||||
candidateSelection = filterBulk(candidateSelection);
|
||||
filterBulk(candidateSelection);
|
||||
candidateSelection = applyCompactionPolicy(candidateSelection, mayUseOffPeak, mayBeStuck);
|
||||
candidateSelection = checkMinFilesCriteria(candidateSelection,
|
||||
comConf.getMinFilesToCompact());
|
||||
|
|
|
@ -16,7 +16,6 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
|
@ -24,9 +23,8 @@ import org.apache.hadoop.hbase.regionserver.StoreUtils;
|
|||
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.base.Predicate;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Collections2;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
|
@ -199,16 +197,9 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
|
|||
|
||||
/**
|
||||
* @param candidates pre-filtrate
|
||||
* @return filtered subset exclude all bulk load files if configured
|
||||
*/
|
||||
protected ArrayList<HStoreFile> filterBulk(ArrayList<HStoreFile> candidates) {
|
||||
candidates.removeAll(Collections2.filter(candidates, new Predicate<HStoreFile>() {
|
||||
@Override
|
||||
public boolean apply(HStoreFile input) {
|
||||
return input.excludeFromMinorCompaction();
|
||||
}
|
||||
}));
|
||||
return candidates;
|
||||
protected void filterBulk(ArrayList<HStoreFile> candidates) {
|
||||
candidates.removeIf(HStoreFile::excludeFromMinorCompaction);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -69,6 +69,11 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
|
|||
return Iterables.filter(files, new Predicate<FileStatus>() {
|
||||
@Override
|
||||
public boolean apply(FileStatus file) {
|
||||
// just for overriding the findbugs NP warnings, as the parameter is marked as Nullable in
|
||||
// the guava Predicate.
|
||||
if (file == null) {
|
||||
return false;
|
||||
}
|
||||
String hfile = file.getPath().getName();
|
||||
boolean foundHFileRefInQueue = hfileRefs.contains(hfile);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
|
|
@ -78,6 +78,11 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
|
|||
return Iterables.filter(files, new Predicate<FileStatus>() {
|
||||
@Override
|
||||
public boolean apply(FileStatus file) {
|
||||
// just for overriding the findbugs NP warnings, as the parameter is marked as Nullable in
|
||||
// the guava Predicate.
|
||||
if (file == null) {
|
||||
return false;
|
||||
}
|
||||
String wal = file.getPath().getName();
|
||||
boolean logInReplicationQueue = wals.contains(wal);
|
||||
if (logInReplicationQueue) {
|
||||
|
|
Loading…
Reference in New Issue