HBASE-25324 Remove unnecessary array to list conversion in SplitLogManager (#2703)
Signed-off-by: Guanghao Zhang <zghao@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org> Signed-off-by: stack <stack@apache.com>
This commit is contained in:
parent
3775464981
commit
e81f49c918
|
@ -148,7 +148,7 @@ public class SplitLogManager {
|
|||
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);
|
||||
}
|
||||
|
||||
|
@ -163,7 +163,7 @@ public class SplitLogManager {
|
|||
* {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem,
|
||||
* Configuration, org.apache.hadoop.hbase.wal.WALFactory)} for tests.
|
||||
*/
|
||||
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<>();
|
||||
|
@ -180,8 +180,8 @@ public class SplitLogManager {
|
|||
Collections.addAll(fileStatus, logfiles);
|
||||
}
|
||||
}
|
||||
FileStatus[] a = new FileStatus[fileStatus.size()];
|
||||
return fileStatus.toArray(a);
|
||||
|
||||
return fileStatus;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -239,11 +239,11 @@ public class SplitLogManager {
|
|||
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.isEmpty()) {
|
||||
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();
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
|
|||
import org.apache.hadoop.hbase.util.CommonFSUtils;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitUtil;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -107,10 +106,10 @@ public class SplitWALManager {
|
|||
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) {
|
||||
|
|
|
@ -29,7 +29,6 @@ import java.util.Map;
|
|||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import org.apache.commons.lang3.ArrayUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -217,17 +216,16 @@ public class WALSplitter {
|
|||
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 (!wals.isEmpty()) {
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue