// could be stuck replaying for ever. Just continue though we
// could have lost some edits.
fs.delete(logfiles[i].getPath(), true);
}
}
ExecutorService threadPool =
Executors.newFixedThreadPool(logWriterThreads);
for (final byte[] key : logEntries.keySet()) {
Thread thread = new Thread(Bytes.toStringBinary(key)) {
@Override
public void run() {
LinkedList<HLogEntry> entries = logEntries.get(key);
LOG.debug("Thread got " + entries.size() + " to process");
long threadTime = System.currentTimeMillis();
try {
int count = 0;
// Items were added to the linkedlist oldest first. Pull them
// out in that order.
for (ListIterator<HLogEntry> i =
entries.listIterator(entries.size());
i.hasPrevious();) {
HLogEntry logEntry = i.previous();
WriterAndPath wap = logWriters.get(key);
if (wap == null) {
Path logfile = new Path(HRegion.getRegionDir(HTableDescriptor
.getTableDir(rootDir, logEntry.getKey().getTablename()),
HRegionInfo.encodeRegionName(key)),
HREGION_OLDLOGFILE_NAME);
Path oldlogfile = null;
SequenceFile.Reader old = null;
if (fs.exists(logfile)) {
FileStatus stat = fs.getFileStatus(logfile);
if (stat.getLen() <= 0) {
LOG.warn("Old hlog file " + logfile + " is zero " +
"length. Deleting existing file");
fs.delete(logfile, false);
} else {
LOG.warn("Old hlog file " + logfile + " already " +
"exists. Copying existing file to new file");
oldlogfile = new Path(logfile.toString() + ".old");
fs.rename(logfile, oldlogfile);
old = new SequenceFile.Reader(fs, oldlogfile, conf);
}
}
SequenceFile.Writer w =
SequenceFile.createWriter(fs, conf, logfile,
getKeyClass(conf), KeyValue.class, getCompressionType(conf));
wap = new WriterAndPath(logfile, w);
logWriters.put(key, wap);
if (LOG.isDebugEnabled()) {
LOG.debug("Creating new hlog file writer for path "
+ logfile + " and region " + Bytes.toStringBinary(key));
}
if (old != null) {
// Copy from existing log file
HLogKey oldkey = newKey(conf);
KeyValue oldval = new KeyValue();
for (; old.next(oldkey, oldval); count++) {
if (LOG.isDebugEnabled() && count > 0
&& count % 10000 == 0) {
LOG.debug("Copied " + count + " edits");
}
w.append(oldkey, oldval);
}
old.close();
fs.delete(oldlogfile, true);
}
}
wap.w.append(logEntry.getKey(), logEntry.getEdit());
count++;
}
if (LOG.isDebugEnabled()) {
LOG.debug("Applied " + count + " total edits to "
+ Bytes.toStringBinary(key) + " in "
+ (System.currentTimeMillis() - threadTime) + "ms");
}
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
LOG.warn("Got while writing region " + Bytes.toStringBinary(key)
+ " log " + e);
e.printStackTrace();
}
}
};
threadPool.execute(thread);
}
threadPool.shutdown();
// Wait for all threads to terminate
try {
for(int i = 0; !threadPool.awaitTermination(5, TimeUnit.SECONDS); i++) {
LOG.debug("Waiting for hlog writers to terminate, iteration #" + i);
}
}catch(InterruptedException ex) {
LOG.warn("Hlog writers were interrupted, possible data loss!");
}