This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new f77bde3 HBASE-22202 Fix new findbugs issues after we upgrade hbase-thirdparty dependencies
f77bde3 is described below
commit f77bde37225d6787d99474ea4b8ac83e0d233a73
Author: Duo Zhang <zhangduo@apache.org>
AuthorDate: Fri Apr 12 11:29:26 2019 +0800
HBASE-22202 Fix new findbugs issues after we upgrade hbase-thirdparty dependencies
---
.../apache/hadoop/hbase/backup/HFileArchiver.java | 38 ++++++++++---------
.../hbase/quotas/NamespaceQuotaSnapshotStore.java | 13 ++-----
.../hbase/quotas/TableQuotaSnapshotStore.java | 16 +++-----
.../hbase/regionserver/StoreFileComparators.java | 43 +++++++++-------------
.../compactions/RatioBasedCompactionPolicy.java | 2 +-
.../compactions/SortedCompactionPolicy.java | 15 ++------
.../master/ReplicationHFileCleaner.java | 5 +++
.../replication/master/ReplicationLogCleaner.java | 5 +++
8 files changed, 61 insertions(+), 76 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
index 0cac629..1783f6b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
@@ -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
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
index b535775..3a97b33 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
@@ -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();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
index c376dfa..9f3fd0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
@@ -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();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java
index 99916d2..5a52adb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java
@@ -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);
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
index e0be6cf..d2ba26a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
@@ -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());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java
index 3eb830a..9b30ab5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java
@@ -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);
}
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
index 7b62169..3271696 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
@@ -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()) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index b9ed8dd..1f3688e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -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) {
|