Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-25324 Remove unnecessary array to list conversion in SplitLogManager #2703

Merged
merged 9 commits into from
Nov 27, 2020
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,7 @@ private SplitLogManagerCoordination getSplitLogManagerCoordination() {
return server.getCoordinatedStateManager().getSplitLogManagerCoordination();
}

private FileStatus[] getFileList(List<Path> logDirs, PathFilter filter) throws IOException {
private List<FileStatus> getFileList(List<Path> logDirs, PathFilter filter) throws IOException {
return getFileList(conf, logDirs, filter);
}

Expand All @@ -166,7 +166,7 @@ private FileStatus[] getFileList(List<Path> logDirs, PathFilter filter) throws I
* Configuration, org.apache.hadoop.hbase.wal.WALFactory)} for tests.
*/
@VisibleForTesting
public static FileStatus[] getFileList(final Configuration conf, final List<Path> logDirs,
public static List<FileStatus> getFileList(final Configuration conf, final List<Path> logDirs,
final PathFilter filter)
throws IOException {
List<FileStatus> fileStatus = new ArrayList<>();
Expand All @@ -183,8 +183,8 @@ public static FileStatus[] getFileList(final Configuration conf, final List<Path
Collections.addAll(fileStatus, logfiles);
}
}
FileStatus[] a = new FileStatus[fileStatus.size()];
return fileStatus.toArray(a);

return fileStatus;
}

/**
Expand Down Expand Up @@ -242,11 +242,11 @@ public long splitLogDistributed(final Set<ServerName> serverNames, final List<Pa
long totalSize = 0;
TaskBatch batch = null;
long startTime = 0;
FileStatus[] logfiles = getFileList(logDirs, filter);
if (logfiles.length != 0) {
List<FileStatus> logfiles = getFileList(logDirs, filter);
if (logfiles.size() != 0) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/logfiles.size != 0/logfiles.isEmpty()/

Copy link
Contributor Author

@yuqi1129 yuqi1129 Nov 25, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice, I will use CollectionUtils.isNotEmpty instead

status.setStatus("Checking directory contents...");
SplitLogCounters.tot_mgr_log_split_batch_start.increment();
LOG.info("Started splitting " + logfiles.length + " logs in " + logDirs +
LOG.info("Started splitting " + logfiles.size() + " logs in " + logDirs +
" for " + serverNames);
startTime = EnvironmentEdgeManager.currentTime();
batch = new TaskBatch();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,10 +108,10 @@ public List<Procedure> splitWALs(ServerName crashedServer, boolean splitMeta)
public List<FileStatus> getWALsToSplit(ServerName serverName, boolean splitMeta)
throws IOException {
List<Path> logDirs = master.getMasterWalManager().getLogDirs(Collections.singleton(serverName));
FileStatus[] fileStatuses =
SplitLogManager.getFileList(this.conf, logDirs, splitMeta ? META_FILTER : NON_META_FILTER);
LOG.info("{} WAL count={}, meta={}", serverName, fileStatuses.length, splitMeta);
return Lists.newArrayList(fileStatuses);
List<FileStatus> fileStatuses =
SplitLogManager.getFileList(this.conf, logDirs, splitMeta ? META_FILTER : NON_META_FILTER);
LOG.info("{} WAL count={}, meta={}", serverName, fileStatuses.size(), splitMeta);
return fileStatuses;
}

private Path getWALSplitDir(ServerName serverName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,6 @@
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
import edu.umd.cs.findbugs.annotations.Nullable;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
Expand All @@ -54,6 +52,7 @@
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WAL.Reader;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -62,7 +61,6 @@
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
import javax.validation.constraints.Null;

/**
* Split RegionServer WAL files. Splits the WAL into new files,
Expand Down Expand Up @@ -220,17 +218,16 @@ public static List<Path> split(Path walRootDir, Path walsDir, Path archiveDir, F
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem rootFS = rootDir.getFileSystem(conf);
WALSplitter splitter = new WALSplitter(factory, conf, walRootDir, walFS, rootDir, rootFS);
final FileStatus[] wals =
final List<FileStatus> wals =
SplitLogManager.getFileList(conf, Collections.singletonList(walsDir), null);
List<Path> splits = new ArrayList<>();
if (ArrayUtils.isNotEmpty(wals)) {
if (CollectionUtils.isNotEmpty(wals)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wals.isEmpty()?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The logic here: if wals is not (empty or null), we will do the following loop, so i can't clearly figure out your meaning, could you description it in detail ?

for (FileStatus wal: wals) {
SplitWALResult splitWALResult = splitter.splitWAL(wal, null);
if (splitWALResult.isFinished()) {
WALSplitUtil.archive(wal.getPath(), splitWALResult.isCorrupt(), archiveDir, walFS, conf);
if (splitter.outputSink.splits != null) {
splits.addAll(splitter.outputSink.splits);
}
//splitter.outputSink.splits is mark as final, do not need null check
splits.addAll(splitter.outputSink.splits);
virajjasani marked this conversation as resolved.
Show resolved Hide resolved
}
}
}
Expand Down