|
@@ -18,15 +18,9 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
-import java.io.BufferedReader;
|
|
|
-import java.io.Closeable;
|
|
|
import java.io.DataOutputStream;
|
|
|
-import java.io.File;
|
|
|
import java.io.FileNotFoundException;
|
|
|
-import java.io.FileOutputStream;
|
|
|
-import java.io.FileReader;
|
|
|
import java.io.IOException;
|
|
|
-import java.io.PrintStream;
|
|
|
import java.text.DateFormat;
|
|
|
import java.text.SimpleDateFormat;
|
|
|
import java.util.Collections;
|
|
@@ -34,7 +28,11 @@ import java.util.Date;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.SortedSet;
|
|
|
import java.util.TreeSet;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.regex.Matcher;
|
|
|
import java.util.regex.Pattern;
|
|
|
|
|
@@ -61,41 +59,43 @@ class BlockPoolSliceScanner {
|
|
|
|
|
|
public static final Log LOG = LogFactory.getLog(BlockPoolSliceScanner.class);
|
|
|
|
|
|
+ private static final String DATA_FORMAT = "yyyy-MM-dd HH:mm:ss,SSS";
|
|
|
+
|
|
|
private static final int MAX_SCAN_RATE = 8 * 1024 * 1024; // 8MB per sec
|
|
|
private static final int MIN_SCAN_RATE = 1 * 1024 * 1024; // 1MB per sec
|
|
|
-
|
|
|
- static final long DEFAULT_SCAN_PERIOD_HOURS = 21*24L; // three weeks
|
|
|
+ private static final long DEFAULT_SCAN_PERIOD_HOURS = 21*24L; // three weeks
|
|
|
+
|
|
|
+ private static final String VERIFICATION_PREFIX = "dncp_block_verification.log";
|
|
|
+
|
|
|
private final String blockPoolId;
|
|
|
-
|
|
|
- private static final String dateFormatString = "yyyy-MM-dd HH:mm:ss,SSS";
|
|
|
-
|
|
|
- static final String verificationLogFile = "dncp_block_verification.log";
|
|
|
- static final int verficationLogLimit = 5; // * numBlocks.
|
|
|
+ private final long scanPeriod;
|
|
|
+ private final AtomicLong lastScanTime = new AtomicLong();
|
|
|
|
|
|
- private long scanPeriod = DEFAULT_SCAN_PERIOD_HOURS * 3600 * 1000;
|
|
|
- private DataNode datanode;
|
|
|
+ private final DataNode datanode;
|
|
|
private final FSDatasetInterface<? extends FSVolumeInterface> dataset;
|
|
|
|
|
|
- // sorted set
|
|
|
- private TreeSet<BlockScanInfo> blockInfoSet;
|
|
|
- private HashMap<Block, BlockScanInfo> blockMap;
|
|
|
+ private final SortedSet<BlockScanInfo> blockInfoSet
|
|
|
+ = new TreeSet<BlockScanInfo>();
|
|
|
+ private final Map<Block, BlockScanInfo> blockMap
|
|
|
+ = new HashMap<Block, BlockScanInfo>();
|
|
|
|
|
|
// processedBlocks keeps track of which blocks are scanned
|
|
|
// since the last run.
|
|
|
- private HashMap<Long, Integer> processedBlocks;
|
|
|
+ private volatile HashMap<Long, Integer> processedBlocks;
|
|
|
|
|
|
private long totalScans = 0;
|
|
|
private long totalScanErrors = 0;
|
|
|
private long totalTransientErrors = 0;
|
|
|
- private long totalBlocksScannedInLastRun = 0; // Used for test only
|
|
|
+ private final AtomicInteger totalBlocksScannedInLastRun = new AtomicInteger(); // Used for test only
|
|
|
|
|
|
private long currentPeriodStart = System.currentTimeMillis();
|
|
|
private long bytesLeft = 0; // Bytes to scan in this period
|
|
|
private long totalBytesToScan = 0;
|
|
|
|
|
|
- private LogFileHandler verificationLog;
|
|
|
+ private final LogFileHandler verificationLog;
|
|
|
|
|
|
- private DataTransferThrottler throttler = null;
|
|
|
+ private final DataTransferThrottler throttler = new DataTransferThrottler(
|
|
|
+ 200, MAX_SCAN_RATE);
|
|
|
|
|
|
private static enum ScanType {
|
|
|
VERIFICATION_SCAN, // scanned as part of periodic verfication
|
|
@@ -133,29 +133,48 @@ class BlockPoolSliceScanner {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- BlockPoolSliceScanner(DataNode datanode,
|
|
|
+ BlockPoolSliceScanner(String bpid, DataNode datanode,
|
|
|
FSDatasetInterface<? extends FSVolumeInterface> dataset,
|
|
|
- Configuration conf, String bpid) {
|
|
|
+ Configuration conf) {
|
|
|
this.datanode = datanode;
|
|
|
this.dataset = dataset;
|
|
|
this.blockPoolId = bpid;
|
|
|
- scanPeriod = conf.getInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY,
|
|
|
+
|
|
|
+ long hours = conf.getInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY,
|
|
|
DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT);
|
|
|
- if ( scanPeriod <= 0 ) {
|
|
|
- scanPeriod = DEFAULT_SCAN_PERIOD_HOURS;
|
|
|
+ if (hours <= 0) {
|
|
|
+ hours = DEFAULT_SCAN_PERIOD_HOURS;
|
|
|
}
|
|
|
- scanPeriod *= 3600 * 1000;
|
|
|
- LOG.info("Periodic Block Verification scan initialized with interval " + scanPeriod + ".");
|
|
|
+ this.scanPeriod = hours * 3600 * 1000;
|
|
|
+ LOG.info("Periodic Block Verification Scanner initialized with interval "
|
|
|
+ + hours + " hours for block pool " + bpid + ".");
|
|
|
+
|
|
|
+ // get the list of blocks and arrange them in random order
|
|
|
+ List<Block> arr = dataset.getFinalizedBlocks(blockPoolId);
|
|
|
+ Collections.shuffle(arr);
|
|
|
+
|
|
|
+ long scanTime = -1;
|
|
|
+ for (Block block : arr) {
|
|
|
+ BlockScanInfo info = new BlockScanInfo( block );
|
|
|
+ info.lastScanTime = scanTime--;
|
|
|
+ //still keep 'info.lastScanType' to NONE.
|
|
|
+ addBlockInfo(info);
|
|
|
+ }
|
|
|
+
|
|
|
+ RollingLogs rollingLogs = null;
|
|
|
+ try {
|
|
|
+ rollingLogs = dataset.createRollingLogs(blockPoolId, VERIFICATION_PREFIX);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Could not open verfication log. " +
|
|
|
+ "Verification times are not stored.");
|
|
|
+ }
|
|
|
+ verificationLog = rollingLogs == null? null: new LogFileHandler(rollingLogs);
|
|
|
}
|
|
|
|
|
|
String getBlockPoolId() {
|
|
|
return blockPoolId;
|
|
|
}
|
|
|
|
|
|
- synchronized boolean isInitialized() {
|
|
|
- return throttler != null;
|
|
|
- }
|
|
|
-
|
|
|
private void updateBytesToScan(long len, long lastScanTime) {
|
|
|
// len could be negative when a block is deleted.
|
|
|
totalBytesToScan += len;
|
|
@@ -197,51 +216,6 @@ class BlockPoolSliceScanner {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void init() throws IOException {
|
|
|
- // get the list of blocks and arrange them in random order
|
|
|
- List<Block> arr = dataset.getFinalizedBlocks(blockPoolId);
|
|
|
- Collections.shuffle(arr);
|
|
|
-
|
|
|
- blockInfoSet = new TreeSet<BlockScanInfo>();
|
|
|
- blockMap = new HashMap<Block, BlockScanInfo>();
|
|
|
-
|
|
|
- long scanTime = -1;
|
|
|
- for (Block block : arr) {
|
|
|
- BlockScanInfo info = new BlockScanInfo( block );
|
|
|
- info.lastScanTime = scanTime--;
|
|
|
- //still keep 'info.lastScanType' to NONE.
|
|
|
- addBlockInfo(info);
|
|
|
- }
|
|
|
-
|
|
|
- /* Pick the first directory that has any existing scanner log.
|
|
|
- * otherwise, pick the first directory.
|
|
|
- */
|
|
|
- File dir = null;
|
|
|
- final List<? extends FSVolumeInterface> volumes = dataset.getVolumes();
|
|
|
- for (FSVolumeInterface vol : volumes) {
|
|
|
- File bpDir = vol.getDirectory(blockPoolId);
|
|
|
- if (LogFileHandler.isFilePresent(bpDir, verificationLogFile)) {
|
|
|
- dir = bpDir;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- if (dir == null) {
|
|
|
- dir = volumes.get(0).getDirectory(blockPoolId);
|
|
|
- }
|
|
|
-
|
|
|
- try {
|
|
|
- // max lines will be updated later during initialization.
|
|
|
- verificationLog = new LogFileHandler(dir, verificationLogFile, 100);
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.warn("Could not open verfication log. " +
|
|
|
- "Verification times are not stored.");
|
|
|
- }
|
|
|
-
|
|
|
- synchronized (this) {
|
|
|
- throttler = new DataTransferThrottler(200, MAX_SCAN_RATE);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private synchronized long getNewBlockScanTime() {
|
|
|
/* If there are a lot of blocks, this returns a random time with in
|
|
|
* the scan period. Otherwise something sooner.
|
|
@@ -255,10 +229,6 @@ class BlockPoolSliceScanner {
|
|
|
|
|
|
/** Adds block to list of blocks */
|
|
|
synchronized void addBlock(ExtendedBlock block) {
|
|
|
- if (!isInitialized()) {
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
BlockScanInfo info = blockMap.get(block.getLocalBlock());
|
|
|
if ( info != null ) {
|
|
|
LOG.warn("Adding an already existing block " + block);
|
|
@@ -274,20 +244,19 @@ class BlockPoolSliceScanner {
|
|
|
|
|
|
/** Deletes the block from internal structures */
|
|
|
synchronized void deleteBlock(Block block) {
|
|
|
- if (!isInitialized()) {
|
|
|
- return;
|
|
|
- }
|
|
|
BlockScanInfo info = blockMap.get(block);
|
|
|
if ( info != null ) {
|
|
|
delBlockInfo(info);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** @return the last scan time */
|
|
|
+ /** @return the last scan time for the block pool. */
|
|
|
+ long getLastScanTime() {
|
|
|
+ return lastScanTime.get();
|
|
|
+ }
|
|
|
+
|
|
|
+ /** @return the last scan time the given block. */
|
|
|
synchronized long getLastScanTime(Block block) {
|
|
|
- if (!isInitialized()) {
|
|
|
- return 0;
|
|
|
- }
|
|
|
BlockScanInfo info = blockMap.get(block);
|
|
|
return info == null? 0: info.lastScanTime;
|
|
|
}
|
|
@@ -302,9 +271,6 @@ class BlockPoolSliceScanner {
|
|
|
private synchronized void updateScanStatus(Block block,
|
|
|
ScanType type,
|
|
|
boolean scanOk) {
|
|
|
- if (!isInitialized()) {
|
|
|
- return;
|
|
|
- }
|
|
|
BlockScanInfo info = blockMap.get(block);
|
|
|
|
|
|
if ( info != null ) {
|
|
@@ -325,9 +291,9 @@ class BlockPoolSliceScanner {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- LogFileHandler log = verificationLog;
|
|
|
- if (log != null) {
|
|
|
- log.appendLine(now, block.getGenerationStamp(), block.getBlockId());
|
|
|
+ if (verificationLog != null) {
|
|
|
+ verificationLog.append(now, block.getGenerationStamp(),
|
|
|
+ block.getBlockId());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -342,6 +308,7 @@ class BlockPoolSliceScanner {
|
|
|
}
|
|
|
|
|
|
static private class LogEntry {
|
|
|
+
|
|
|
long blockId = -1;
|
|
|
long verificationTime = -1;
|
|
|
long genStamp = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
|
|
@@ -355,6 +322,14 @@ class BlockPoolSliceScanner {
|
|
|
private static Pattern entryPattern =
|
|
|
Pattern.compile("\\G\\s*([^=\\p{Space}]+)=\"(.*?)\"\\s*");
|
|
|
|
|
|
+ static String toString(long verificationTime, long genStamp, long blockId,
|
|
|
+ DateFormat dateFormat) {
|
|
|
+ return "\ndate=\"" + dateFormat.format(new Date(verificationTime))
|
|
|
+ + "\"\t time=\"" + verificationTime
|
|
|
+ + "\"\t genstamp=\"" + genStamp
|
|
|
+ + "\"\t id=\"" + blockId + "\"";
|
|
|
+ }
|
|
|
+
|
|
|
static LogEntry parseEntry(String line) {
|
|
|
LogEntry entry = new LogEntry();
|
|
|
|
|
@@ -491,8 +466,8 @@ class BlockPoolSliceScanner {
|
|
|
}
|
|
|
|
|
|
// Used for tests only
|
|
|
- long getBlocksScannedInLastRun() {
|
|
|
- return totalBlocksScannedInLastRun;
|
|
|
+ int getBlocksScannedInLastRun() {
|
|
|
+ return totalBlocksScannedInLastRun.get();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -503,33 +478,19 @@ class BlockPoolSliceScanner {
|
|
|
* to exit.
|
|
|
*/
|
|
|
private boolean assignInitialVerificationTimes() {
|
|
|
- int numBlocks = 1;
|
|
|
- LogFileHandler log = null;
|
|
|
- synchronized (this) {
|
|
|
- log = verificationLog;
|
|
|
- numBlocks = Math.max(blockMap.size(), 1);
|
|
|
- }
|
|
|
-
|
|
|
- long now = System.currentTimeMillis();
|
|
|
- LogFileHandler.Reader logReader[] = new LogFileHandler.Reader[2];
|
|
|
- try {
|
|
|
- if (log != null) {
|
|
|
- logReader[0] = log.getCurrentFileReader();
|
|
|
- logReader[1] = log.getPreviousFileReader();
|
|
|
- }
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.warn("Could not read previous verification times", e);
|
|
|
- }
|
|
|
-
|
|
|
- try {
|
|
|
- for (LogFileHandler.Reader reader : logReader) {
|
|
|
- // update verification times from the verificationLog.
|
|
|
- while (logReader != null && reader.hasNext()) {
|
|
|
+ //First updates the last verification times from the log file.
|
|
|
+ if (verificationLog != null) {
|
|
|
+ long now = System.currentTimeMillis();
|
|
|
+ RollingLogs.LineIterator logIterator = null;
|
|
|
+ try {
|
|
|
+ logIterator = verificationLog.logs.iterator(false);
|
|
|
+ // update verification times from the verificationLog.
|
|
|
+ while (logIterator.hasNext()) {
|
|
|
if (!datanode.shouldRun
|
|
|
|| datanode.blockScanner.blockScannerThread.isInterrupted()) {
|
|
|
return false;
|
|
|
}
|
|
|
- LogEntry entry = LogEntry.parseEntry(reader.next());
|
|
|
+ LogEntry entry = LogEntry.parseEntry(logIterator.next());
|
|
|
if (entry != null) {
|
|
|
updateBlockInfo(entry);
|
|
|
if (now - entry.verificationTime < scanPeriod) {
|
|
@@ -540,35 +501,35 @@ class BlockPoolSliceScanner {
|
|
|
updateBytesLeft(-info.block.getNumBytes());
|
|
|
processedBlocks.put(entry.blockId, 1);
|
|
|
}
|
|
|
- if (reader.file == log.prevFile) {
|
|
|
+ if (logIterator.isPrevious()) {
|
|
|
// write the log entry to current file
|
|
|
// so that the entry is preserved for later runs.
|
|
|
- log.appendLine(entry.verificationTime, entry.genStamp,
|
|
|
+ verificationLog.append(entry.verificationTime, entry.genStamp,
|
|
|
entry.blockId);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Failed to read previous verification times.", e);
|
|
|
+ } finally {
|
|
|
+ IOUtils.closeStream(logIterator);
|
|
|
}
|
|
|
- } finally {
|
|
|
- IOUtils.closeStream(logReader[0]);
|
|
|
- IOUtils.closeStream(logReader[1]);
|
|
|
}
|
|
|
|
|
|
- /* Initially spread the block reads over half of
|
|
|
- * MIN_SCAN_PERIOD so that we don't keep scanning the
|
|
|
- * blocks too quickly when restarted.
|
|
|
- */
|
|
|
- long verifyInterval = (long) (Math.min( scanPeriod/2.0/numBlocks,
|
|
|
- 10*60*1000 ));
|
|
|
- long lastScanTime = System.currentTimeMillis() - scanPeriod;
|
|
|
|
|
|
/* Before this loop, entries in blockInfoSet that are not
|
|
|
* updated above have lastScanTime of <= 0 . Loop until first entry has
|
|
|
* lastModificationTime > 0.
|
|
|
*/
|
|
|
synchronized (this) {
|
|
|
+ final int numBlocks = Math.max(blockMap.size(), 1);
|
|
|
+ // Initially spread the block reads over half of scan period
|
|
|
+ // so that we don't keep scanning the blocks too quickly when restarted.
|
|
|
+ long verifyInterval = Math.min(scanPeriod/(2L * numBlocks), 10*60*1000L);
|
|
|
+ long lastScanTime = System.currentTimeMillis() - scanPeriod;
|
|
|
+
|
|
|
if (!blockInfoSet.isEmpty()) {
|
|
|
BlockScanInfo info;
|
|
|
while ((info = blockInfoSet.first()).lastScanTime < 0) {
|
|
@@ -586,11 +547,6 @@ class BlockPoolSliceScanner {
|
|
|
private synchronized void updateBytesLeft(long len) {
|
|
|
bytesLeft += len;
|
|
|
}
|
|
|
-
|
|
|
- static File getCurrentFile(FSVolumeInterface vol, String bpid) throws IOException {
|
|
|
- return LogFileHandler.getCurrentFile(vol.getDirectory(bpid),
|
|
|
- BlockPoolSliceScanner.verificationLogFile);
|
|
|
- }
|
|
|
|
|
|
private synchronized void startNewPeriod() {
|
|
|
LOG.info("Starting a new period : work left in prev period : "
|
|
@@ -604,26 +560,21 @@ class BlockPoolSliceScanner {
|
|
|
|
|
|
void scanBlockPoolSlice() {
|
|
|
startNewPeriod();
|
|
|
- if (processedBlocks != null) {
|
|
|
- totalBlocksScannedInLastRun = processedBlocks.size();
|
|
|
- }
|
|
|
// Create a new processedBlocks structure
|
|
|
processedBlocks = new HashMap<Long, Integer>();
|
|
|
- if (verificationLog != null) {
|
|
|
- try {
|
|
|
- verificationLog.openCurFile();
|
|
|
- } catch (FileNotFoundException ex) {
|
|
|
- LOG.warn("Could not open current file");
|
|
|
- }
|
|
|
- }
|
|
|
if (!assignInitialVerificationTimes()) {
|
|
|
return;
|
|
|
}
|
|
|
// Start scanning
|
|
|
- scan();
|
|
|
+ try {
|
|
|
+ scan();
|
|
|
+ } finally {
|
|
|
+ totalBlocksScannedInLastRun.set(processedBlocks.size());
|
|
|
+ lastScanTime.set(System.currentTimeMillis());
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- public void scan() {
|
|
|
+ private void scan() {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Starting to scan blockpool: " + blockPoolId);
|
|
|
}
|
|
@@ -663,7 +614,7 @@ class BlockPoolSliceScanner {
|
|
|
private synchronized void cleanUp() {
|
|
|
if (verificationLog != null) {
|
|
|
try {
|
|
|
- verificationLog.roll();
|
|
|
+ verificationLog.logs.roll();
|
|
|
} catch (IOException ex) {
|
|
|
LOG.warn("Received exception: ", ex);
|
|
|
verificationLog.close();
|
|
@@ -686,7 +637,7 @@ class BlockPoolSliceScanner {
|
|
|
int inScanPeriod = 0;
|
|
|
int neverScanned = 0;
|
|
|
|
|
|
- DateFormat dateFormat = new SimpleDateFormat(dateFormatString);
|
|
|
+ DateFormat dateFormat = new SimpleDateFormat(DATA_FORMAT);
|
|
|
|
|
|
int total = blockInfoSet.size();
|
|
|
|
|
@@ -751,191 +702,33 @@ class BlockPoolSliceScanner {
|
|
|
|
|
|
/**
|
|
|
* This class takes care of log file used to store the last verification
|
|
|
- * times of the blocks. It rolls the current file when it is too big etc.
|
|
|
- * If there is an error while writing, it stops updating with an error
|
|
|
- * message.
|
|
|
+ * times of the blocks.
|
|
|
*/
|
|
|
private static class LogFileHandler {
|
|
|
-
|
|
|
- private static final String curFileSuffix = ".curr";
|
|
|
- private static final String prevFileSuffix = ".prev";
|
|
|
- private final DateFormat dateFormat = new SimpleDateFormat(dateFormatString);
|
|
|
-
|
|
|
- static File getCurrentFile(File dir, String filePrefix) {
|
|
|
- return new File(dir, filePrefix + curFileSuffix);
|
|
|
- }
|
|
|
-
|
|
|
- public Reader getPreviousFileReader() throws IOException {
|
|
|
- return new Reader(prevFile);
|
|
|
- }
|
|
|
-
|
|
|
- public Reader getCurrentFileReader() throws IOException {
|
|
|
- return new Reader(curFile);
|
|
|
- }
|
|
|
-
|
|
|
- static boolean isFilePresent(File dir, String filePrefix) {
|
|
|
- return new File(dir, filePrefix + curFileSuffix).exists() ||
|
|
|
- new File(dir, filePrefix + prevFileSuffix).exists();
|
|
|
- }
|
|
|
- private File curFile;
|
|
|
- private File prevFile;
|
|
|
-
|
|
|
- private PrintStream out;
|
|
|
-
|
|
|
- /**
|
|
|
- * Opens the log file for appending.
|
|
|
- * Note that rolling will happen only after "updateLineCount()" is
|
|
|
- * called. This is so that line count could be updated in a separate
|
|
|
- * thread without delaying start up.
|
|
|
- *
|
|
|
- * @param dir where the logs files are located.
|
|
|
- * @param filePrefix prefix of the file.
|
|
|
- * @param maxNumLines max lines in a file (its a soft limit).
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- LogFileHandler(File dir, String filePrefix, int maxNumLines)
|
|
|
- throws IOException {
|
|
|
- curFile = new File(dir, filePrefix + curFileSuffix);
|
|
|
- prevFile = new File(dir, filePrefix + prevFileSuffix);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Append "\n" + line.
|
|
|
- * If the log file need to be rolled, it will done after
|
|
|
- * appending the text.
|
|
|
- * This does not throw IOException when there is an error while
|
|
|
- * appending. Currently does not throw an error even if rolling
|
|
|
- * fails (may be it should?).
|
|
|
- * return true if append was successful.
|
|
|
- */
|
|
|
- synchronized boolean appendLine(String line) {
|
|
|
- if (out == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- out.println();
|
|
|
- out.print(line);
|
|
|
- return true;
|
|
|
- }
|
|
|
-
|
|
|
- boolean appendLine(long verificationTime, long genStamp, long blockId) {
|
|
|
- return appendLine("date=\""
|
|
|
- + dateFormat.format(new Date(verificationTime)) + "\"\t " + "time=\""
|
|
|
- + verificationTime + "\"\t " + "genstamp=\"" + genStamp + "\"\t "
|
|
|
- + "id=\"" + blockId + "\"");
|
|
|
- }
|
|
|
-
|
|
|
- private synchronized void openCurFile() throws FileNotFoundException {
|
|
|
- close();
|
|
|
- out = new PrintStream(new FileOutputStream(curFile, true));
|
|
|
- }
|
|
|
-
|
|
|
- private void roll() throws IOException {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Rolling current file: " + curFile.getAbsolutePath()
|
|
|
- + " to previous file: " + prevFile.getAbsolutePath());
|
|
|
- }
|
|
|
+ private final DateFormat dateFormat = new SimpleDateFormat(DATA_FORMAT);
|
|
|
|
|
|
- if (!prevFile.delete() && prevFile.exists()) {
|
|
|
- throw new IOException("Could not delete " + prevFile);
|
|
|
- }
|
|
|
-
|
|
|
- close();
|
|
|
+ private final RollingLogs logs;
|
|
|
|
|
|
- if (!curFile.renameTo(prevFile)) {
|
|
|
- throw new IOException("Could not rename " + curFile +
|
|
|
- " to " + prevFile);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- synchronized void close() {
|
|
|
- if (out != null) {
|
|
|
- out.close();
|
|
|
- out = null;
|
|
|
- }
|
|
|
+ private LogFileHandler(RollingLogs logs) {
|
|
|
+ this.logs = logs;
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * This is used to read the lines in order.
|
|
|
- * If the data is not read completely (i.e, untill hasNext() returns
|
|
|
- * false), it needs to be explicitly
|
|
|
- */
|
|
|
- private static class Reader implements Iterator<String>, Closeable {
|
|
|
-
|
|
|
- BufferedReader reader;
|
|
|
- File file;
|
|
|
- String line;
|
|
|
- boolean closed = false;
|
|
|
-
|
|
|
- private Reader(File file) throws IOException {
|
|
|
- reader = null;
|
|
|
- this.file = file;
|
|
|
- readNext();
|
|
|
- }
|
|
|
-
|
|
|
- private boolean openFile() throws IOException {
|
|
|
- if (file == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- if (reader != null ) {
|
|
|
- reader.close();
|
|
|
- reader = null;
|
|
|
- }
|
|
|
- if (file.exists()) {
|
|
|
- reader = new BufferedReader(new FileReader(file));
|
|
|
- return true;
|
|
|
- } else {
|
|
|
- return false;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // read next line if possible.
|
|
|
- private void readNext() throws IOException {
|
|
|
- line = null;
|
|
|
- if (reader == null) {
|
|
|
- openFile();
|
|
|
- }
|
|
|
- try {
|
|
|
- if (reader != null && (line = reader.readLine()) != null) {
|
|
|
- return;
|
|
|
- }
|
|
|
- } finally {
|
|
|
- if (!hasNext()) {
|
|
|
- close();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public boolean hasNext() {
|
|
|
- return line != null;
|
|
|
- }
|
|
|
-
|
|
|
- public String next() {
|
|
|
- String curLine = line;
|
|
|
- try {
|
|
|
- readNext();
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.info("Could not read next line in LogHandler", e);
|
|
|
- }
|
|
|
- return curLine;
|
|
|
- }
|
|
|
|
|
|
- public void remove() {
|
|
|
- throw new RuntimeException("remove() is not supported.");
|
|
|
+ void append(long verificationTime, long genStamp, long blockId) {
|
|
|
+ final String m = LogEntry.toString(verificationTime, genStamp, blockId,
|
|
|
+ dateFormat);
|
|
|
+ try {
|
|
|
+ logs.appender().append(m);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Failed to append to " + logs + ", m=" + m, e);
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- public void close() throws IOException {
|
|
|
- if (!closed) {
|
|
|
- try {
|
|
|
- if (reader != null) {
|
|
|
- reader.close();
|
|
|
- }
|
|
|
- } finally {
|
|
|
- file = null;
|
|
|
- reader = null;
|
|
|
- closed = true;
|
|
|
- }
|
|
|
- }
|
|
|
+ void close() {
|
|
|
+ try {
|
|
|
+ logs.appender().close();
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Failed to close the appender of " + logs, e);
|
|
|
}
|
|
|
- }
|
|
|
+ }
|
|
|
}
|
|
|
}
|