Path dest = new Path(fullyQualifiedFileName + ".recovered");
log.debug("Sorting log file to DFS " + dest);
fs.mkdirs(dest);
int part = 0;
Reader reader = new SequenceFile.Reader(local, new Path(findLocalFilename(localLog)), fs.getConf());
try {
final ArrayList<Pair<LogFileKey,LogFileValue>> kv = new ArrayList<Pair<LogFileKey,LogFileValue>>();
long memorySize = 0;
while (true) {
final long position = reader.getPosition();
final LogFileKey key = new LogFileKey();
final LogFileValue value = new LogFileValue();
try {
if (!reader.next(key, value))
break;
} catch (EOFException e) {
log.warn("Unexpected end of file reading write ahead log " + localLog);
break;
}
kv.add(new Pair<LogFileKey,LogFileValue>(key, value));
memorySize += reader.getPosition() - position;
if (memorySize > SORT_BUFFER_SIZE) {
writeSortedEntries(dest, part++, kv);
kv.clear();
memorySize = 0;
}
}
if (!kv.isEmpty())
writeSortedEntries(dest, part++, kv);
fs.create(new Path(dest, "finished")).close();
} finally {
reader.close();
}
}
private void writeSortedEntries(Path dest, int part, final List<Pair<LogFileKey,LogFileValue>> kv) throws IOException {
String path = dest + String.format("/part-r-%05d", part);
log.debug("Writing partial log file to DFS " + path);
log.debug("Sorting");
Span span = Trace.start("Logger sort");
span.data("logfile", dest.getName());
Collections.sort(kv, new Comparator<Pair<LogFileKey,LogFileValue>>() {
@Override
public int compare(Pair<LogFileKey,LogFileValue> o1, Pair<LogFileKey,LogFileValue> o2) {
return o1.getFirst().compareTo(o2.getFirst());
}
});
span.stop();
span = Trace.start("Logger write");
span.data("logfile", dest.getName());
MapFile.Writer writer = new MapFile.Writer(fs.getConf(), fs, path, LogFileKey.class, LogFileValue.class);
short replication = (short) acuConf.getCount(Property.LOGGER_RECOVERY_FILE_REPLICATION);
fs.setReplication(new Path(path + "/" + MapFile.DATA_FILE_NAME), replication);
fs.setReplication(new Path(path + "/" + MapFile.INDEX_FILE_NAME), replication);
try {
for (Pair<LogFileKey,LogFileValue> entry : kv)
writer.append(entry.getFirst(), entry.getSecond());
} finally {
writer.close();
span.stop();
}
}
private void copyLog(final String localLog, final String fullyQualifiedFileName) throws IOException {
Path dest = new Path(fullyQualifiedFileName + ".copy");
log.debug("Copying log file to DFS " + dest);
fs.delete(dest, true);
LogFileKey key = new LogFileKey();
LogFileValue value = new LogFileValue();
Writer writer = null;
Reader reader = null;
try {
short replication = (short) acuConf.getCount(Property.LOGGER_RECOVERY_FILE_REPLICATION);
writer = SequenceFile.createWriter(fs, fs.getConf(), dest, LogFileKey.class, LogFileValue.class, fs.getConf().getInt("io.file.buffer.size", 4096),
replication, fs.getDefaultBlockSize(), SequenceFile.CompressionType.BLOCK, new DefaultCodec(), null, new Metadata());
FileSystem local = TraceFileSystem.wrap(FileSystem.getLocal(fs.getConf()).getRaw());
reader = new SequenceFile.Reader(local, new Path(findLocalFilename(localLog)), fs.getConf());
while (reader.next(key, value)) {
writer.append(key, value);
}
} catch (IOException ex) {
log.warn("May have a partial copy of a recovery file: " + localLog, ex);
} finally {
if (reader != null)
reader.close();
if (writer != null)
writer.close();
}
// Make file appear in the shared file system as the target name only after it is completely copied
fs.rename(dest, new Path(fullyQualifiedFileName));