001 /**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements. See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership. The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License. You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018 package org.apache.hadoop.hdfs.qjournal.server;
019
020 import java.io.Closeable;
021 import java.io.File;
022 import java.io.FileInputStream;
023 import java.io.IOException;
024 import java.io.InputStream;
025 import java.io.OutputStreamWriter;
026 import java.net.URL;
027 import java.security.PrivilegedExceptionAction;
028 import java.util.Iterator;
029 import java.util.List;
030 import java.util.concurrent.TimeUnit;
031
032 import org.apache.commons.logging.Log;
033 import org.apache.commons.logging.LogFactory;
034 import org.apache.hadoop.conf.Configuration;
035 import org.apache.hadoop.fs.FileUtil;
036 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
037 import org.apache.hadoop.hdfs.qjournal.protocol.JournalNotFormattedException;
038 import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
039 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
040 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
041 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
042 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PersistedRecoveryPaxosData;
043 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
044 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
045 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
046 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
047 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
048 import org.apache.hadoop.hdfs.server.common.StorageInfo;
049 import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
050 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
051 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
052 import org.apache.hadoop.hdfs.server.namenode.JournalManager;
053 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
054 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
055 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
056 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
057 import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
058 import org.apache.hadoop.hdfs.util.BestEffortLongFile;
059 import org.apache.hadoop.hdfs.util.PersistentLongFile;
060 import org.apache.hadoop.io.IOUtils;
061 import org.apache.hadoop.ipc.Server;
062 import org.apache.hadoop.security.SecurityUtil;
063 import org.apache.hadoop.security.UserGroupInformation;
064
065 import com.google.common.annotations.VisibleForTesting;
066 import com.google.common.base.Charsets;
067 import com.google.common.base.Preconditions;
068 import com.google.common.base.Stopwatch;
069 import com.google.common.collect.ImmutableList;
070 import com.google.common.collect.Range;
071 import com.google.common.collect.Ranges;
072 import com.google.protobuf.TextFormat;
073
074 /**
075 * A JournalNode can manage journals for several clusters at once.
076 * Each such journal is entirely independent despite being hosted by
077 * the same JVM.
078 */
079 public class Journal implements Closeable {
080 static final Log LOG = LogFactory.getLog(Journal.class);
081
082
083 // Current writing state
084 private EditLogOutputStream curSegment;
085 private long curSegmentTxId = HdfsConstants.INVALID_TXID;
086 private long nextTxId = HdfsConstants.INVALID_TXID;
087 private long highestWrittenTxId = 0;
088
089 private final String journalId;
090
091 private final JNStorage storage;
092
093 /**
094 * When a new writer comes along, it asks each node to promise
095 * to ignore requests from any previous writer, as identified
096 * by epoch number. In order to make such a promise, the epoch
097 * number of that writer is stored persistently on disk.
098 */
099 private PersistentLongFile lastPromisedEpoch;
100
101 /**
102 * Each IPC that comes from a given client contains a serial number
103 * which only increases from the client's perspective. Whenever
104 * we switch epochs, we reset this back to -1. Whenever an IPC
105 * comes from a client, we ensure that it is strictly higher
106 * than any previous IPC. This guards against any bugs in the IPC
107 * layer that would re-order IPCs or cause a stale retry from an old
108 * request to resurface and confuse things.
109 */
110 private long currentEpochIpcSerial = -1;
111
112 /**
113 * The epoch number of the last writer to actually write a transaction.
114 * This is used to differentiate log segments after a crash at the very
115 * beginning of a segment. See the the 'testNewerVersionOfSegmentWins'
116 * test case.
117 */
118 private PersistentLongFile lastWriterEpoch;
119
120 /**
121 * Lower-bound on the last committed transaction ID. This is not
122 * depended upon for correctness, but acts as a sanity check
123 * during the recovery procedures, and as a visibility mark
124 * for clients reading in-progress logs.
125 */
126 private BestEffortLongFile committedTxnId;
127
128 public static final String LAST_PROMISED_FILENAME = "last-promised-epoch";
129 public static final String LAST_WRITER_EPOCH = "last-writer-epoch";
130 private static final String COMMITTED_TXID_FILENAME = "committed-txid";
131
132 private final FileJournalManager fjm;
133
134 private final JournalMetrics metrics;
135
136 /**
137 * Time threshold for sync calls, beyond which a warning should be logged to the console.
138 */
139 private static final int WARN_SYNC_MILLIS_THRESHOLD = 1000;
140
141 Journal(Configuration conf, File logDir, String journalId,
142 StartupOption startOpt, StorageErrorReporter errorReporter)
143 throws IOException {
144 storage = new JNStorage(conf, logDir, startOpt, errorReporter);
145 this.journalId = journalId;
146
147 refreshCachedData();
148
149 this.fjm = storage.getJournalManager();
150
151 this.metrics = JournalMetrics.create(this);
152
153 EditLogFile latest = scanStorageForLatestEdits();
154 if (latest != null) {
155 highestWrittenTxId = latest.getLastTxId();
156 }
157 }
158
159 /**
160 * Reload any data that may have been cached. This is necessary
161 * when we first load the Journal, but also after any formatting
162 * operation, since the cached data is no longer relevant.
163 */
164 private synchronized void refreshCachedData() {
165 IOUtils.closeStream(committedTxnId);
166
167 File currentDir = storage.getSingularStorageDir().getCurrentDir();
168 this.lastPromisedEpoch = new PersistentLongFile(
169 new File(currentDir, LAST_PROMISED_FILENAME), 0);
170 this.lastWriterEpoch = new PersistentLongFile(
171 new File(currentDir, LAST_WRITER_EPOCH), 0);
172 this.committedTxnId = new BestEffortLongFile(
173 new File(currentDir, COMMITTED_TXID_FILENAME),
174 HdfsConstants.INVALID_TXID);
175 }
176
177 /**
178 * Scan the local storage directory, and return the segment containing
179 * the highest transaction.
180 * @return the EditLogFile with the highest transactions, or null
181 * if no files exist.
182 */
183 private synchronized EditLogFile scanStorageForLatestEdits() throws IOException {
184 if (!fjm.getStorageDirectory().getCurrentDir().exists()) {
185 return null;
186 }
187
188 LOG.info("Scanning storage " + fjm);
189 List<EditLogFile> files = fjm.getLogFiles(0);
190
191 while (!files.isEmpty()) {
192 EditLogFile latestLog = files.remove(files.size() - 1);
193 latestLog.scanLog();
194 LOG.info("Latest log is " + latestLog);
195 if (latestLog.getLastTxId() == HdfsConstants.INVALID_TXID) {
196 // the log contains no transactions
197 LOG.warn("Latest log " + latestLog + " has no transactions. " +
198 "moving it aside and looking for previous log");
199 latestLog.moveAsideEmptyFile();
200 } else {
201 return latestLog;
202 }
203 }
204
205 LOG.info("No files in " + fjm);
206 return null;
207 }
208
209 /**
210 * Format the local storage with the given namespace.
211 */
212 void format(NamespaceInfo nsInfo) throws IOException {
213 Preconditions.checkState(nsInfo.getNamespaceID() != 0,
214 "can't format with uninitialized namespace info: %s",
215 nsInfo);
216 LOG.info("Formatting " + this + " with namespace info: " +
217 nsInfo);
218 storage.format(nsInfo);
219 refreshCachedData();
220 }
221
222 /**
223 * Unlock and release resources.
224 */
225 @Override // Closeable
226 public void close() throws IOException {
227 storage.close();
228 IOUtils.closeStream(committedTxnId);
229 IOUtils.closeStream(curSegment);
230 }
231
232 JNStorage getStorage() {
233 return storage;
234 }
235
236 String getJournalId() {
237 return journalId;
238 }
239
240 /**
241 * @return the last epoch which this node has promised not to accept
242 * any lower epoch, or 0 if no promises have been made.
243 */
244 synchronized long getLastPromisedEpoch() throws IOException {
245 checkFormatted();
246 return lastPromisedEpoch.get();
247 }
248
249 synchronized public long getLastWriterEpoch() throws IOException {
250 checkFormatted();
251 return lastWriterEpoch.get();
252 }
253
254 synchronized long getCommittedTxnIdForTests() throws IOException {
255 return committedTxnId.get();
256 }
257
258 synchronized long getCurrentLagTxns() throws IOException {
259 long committed = committedTxnId.get();
260 if (committed == 0) {
261 return 0;
262 }
263
264 return Math.max(committed - highestWrittenTxId, 0L);
265 }
266
267 synchronized long getHighestWrittenTxId() {
268 return highestWrittenTxId;
269 }
270
271 @VisibleForTesting
272 JournalMetrics getMetricsForTests() {
273 return metrics;
274 }
275
276 /**
277 * Try to create a new epoch for this journal.
278 * @param nsInfo the namespace, which is verified for consistency or used to
279 * format, if the Journal has not yet been written to.
280 * @param epoch the epoch to start
281 * @return the status information necessary to begin recovery
282 * @throws IOException if the node has already made a promise to another
283 * writer with a higher epoch number, if the namespace is inconsistent,
284 * or if a disk error occurs.
285 */
286 synchronized NewEpochResponseProto newEpoch(
287 NamespaceInfo nsInfo, long epoch) throws IOException {
288
289 checkFormatted();
290 storage.checkConsistentNamespace(nsInfo);
291
292 // Check that the new epoch being proposed is in fact newer than
293 // any other that we've promised.
294 if (epoch <= getLastPromisedEpoch()) {
295 throw new IOException("Proposed epoch " + epoch + " <= last promise " +
296 getLastPromisedEpoch());
297 }
298
299 updateLastPromisedEpoch(epoch);
300 abortCurSegment();
301
302 NewEpochResponseProto.Builder builder =
303 NewEpochResponseProto.newBuilder();
304
305 EditLogFile latestFile = scanStorageForLatestEdits();
306
307 if (latestFile != null) {
308 builder.setLastSegmentTxId(latestFile.getFirstTxId());
309 }
310
311 return builder.build();
312 }
313
314 private void updateLastPromisedEpoch(long newEpoch) throws IOException {
315 LOG.info("Updating lastPromisedEpoch from " + lastPromisedEpoch.get() +
316 " to " + newEpoch + " for client " + Server.getRemoteIp());
317 lastPromisedEpoch.set(newEpoch);
318
319 // Since we have a new writer, reset the IPC serial - it will start
320 // counting again from 0 for this writer.
321 currentEpochIpcSerial = -1;
322 }
323
324 private void abortCurSegment() throws IOException {
325 if (curSegment == null) {
326 return;
327 }
328
329 curSegment.abort();
330 curSegment = null;
331 curSegmentTxId = HdfsConstants.INVALID_TXID;
332 }
333
334 /**
335 * Write a batch of edits to the journal.
336 * {@see QJournalProtocol#journal(RequestInfo, long, long, int, byte[])}
337 */
338 synchronized void journal(RequestInfo reqInfo,
339 long segmentTxId, long firstTxnId,
340 int numTxns, byte[] records) throws IOException {
341 checkFormatted();
342 checkWriteRequest(reqInfo);
343
344 checkSync(curSegment != null,
345 "Can't write, no segment open");
346
347 if (curSegmentTxId != segmentTxId) {
348 // Sanity check: it is possible that the writer will fail IPCs
349 // on both the finalize() and then the start() of the next segment.
350 // This could cause us to continue writing to an old segment
351 // instead of rolling to a new one, which breaks one of the
352 // invariants in the design. If it happens, abort the segment
353 // and throw an exception.
354 JournalOutOfSyncException e = new JournalOutOfSyncException(
355 "Writer out of sync: it thinks it is writing segment " + segmentTxId
356 + " but current segment is " + curSegmentTxId);
357 abortCurSegment();
358 throw e;
359 }
360
361 checkSync(nextTxId == firstTxnId,
362 "Can't write txid " + firstTxnId + " expecting nextTxId=" + nextTxId);
363
364 long lastTxnId = firstTxnId + numTxns - 1;
365 if (LOG.isTraceEnabled()) {
366 LOG.trace("Writing txid " + firstTxnId + "-" + lastTxnId);
367 }
368
369 // If the edit has already been marked as committed, we know
370 // it has been fsynced on a quorum of other nodes, and we are
371 // "catching up" with the rest. Hence we do not need to fsync.
372 boolean isLagging = lastTxnId <= committedTxnId.get();
373 boolean shouldFsync = !isLagging;
374
375 curSegment.writeRaw(records, 0, records.length);
376 curSegment.setReadyToFlush();
377 Stopwatch sw = new Stopwatch();
378 sw.start();
379 curSegment.flush(shouldFsync);
380 sw.stop();
381
382 metrics.addSync(sw.elapsedTime(TimeUnit.MICROSECONDS));
383 if (sw.elapsedTime(TimeUnit.MILLISECONDS) > WARN_SYNC_MILLIS_THRESHOLD) {
384 LOG.warn("Sync of transaction range " + firstTxnId + "-" + lastTxnId +
385 " took " + sw.elapsedTime(TimeUnit.MILLISECONDS) + "ms");
386 }
387
388 if (isLagging) {
389 // This batch of edits has already been committed on a quorum of other
390 // nodes. So, we are in "catch up" mode. This gets its own metric.
391 metrics.batchesWrittenWhileLagging.incr(1);
392 }
393
394 metrics.batchesWritten.incr(1);
395 metrics.bytesWritten.incr(records.length);
396 metrics.txnsWritten.incr(numTxns);
397
398 highestWrittenTxId = lastTxnId;
399 nextTxId = lastTxnId + 1;
400 }
401
402 public void heartbeat(RequestInfo reqInfo) throws IOException {
403 checkRequest(reqInfo);
404 }
405
406 /**
407 * Ensure that the given request is coming from the correct writer and in-order.
408 * @param reqInfo the request info
409 * @throws IOException if the request is invalid.
410 */
411 private synchronized void checkRequest(RequestInfo reqInfo) throws IOException {
412 // Invariant 25 from ZAB paper
413 if (reqInfo.getEpoch() < lastPromisedEpoch.get()) {
414 throw new IOException("IPC's epoch " + reqInfo.getEpoch() +
415 " is less than the last promised epoch " +
416 lastPromisedEpoch.get());
417 } else if (reqInfo.getEpoch() > lastPromisedEpoch.get()) {
418 // A newer client has arrived. Fence any previous writers by updating
419 // the promise.
420 updateLastPromisedEpoch(reqInfo.getEpoch());
421 }
422
423 // Ensure that the IPCs are arriving in-order as expected.
424 checkSync(reqInfo.getIpcSerialNumber() > currentEpochIpcSerial,
425 "IPC serial %s from client %s was not higher than prior highest " +
426 "IPC serial %s", reqInfo.getIpcSerialNumber(),
427 Server.getRemoteIp(),
428 currentEpochIpcSerial);
429 currentEpochIpcSerial = reqInfo.getIpcSerialNumber();
430
431 if (reqInfo.hasCommittedTxId()) {
432 Preconditions.checkArgument(
433 reqInfo.getCommittedTxId() >= committedTxnId.get(),
434 "Client trying to move committed txid backward from " +
435 committedTxnId.get() + " to " + reqInfo.getCommittedTxId());
436
437 committedTxnId.set(reqInfo.getCommittedTxId());
438 }
439 }
440
441 private synchronized void checkWriteRequest(RequestInfo reqInfo) throws IOException {
442 checkRequest(reqInfo);
443
444 if (reqInfo.getEpoch() != lastWriterEpoch.get()) {
445 throw new IOException("IPC's epoch " + reqInfo.getEpoch() +
446 " is not the current writer epoch " +
447 lastWriterEpoch.get());
448 }
449 }
450
451 public synchronized boolean isFormatted() {
452 return storage.isFormatted();
453 }
454
455 private void checkFormatted() throws JournalNotFormattedException {
456 if (!isFormatted()) {
457 throw new JournalNotFormattedException("Journal " +
458 storage.getSingularStorageDir() + " not formatted");
459 }
460 }
461
462 /**
463 * @throws JournalOutOfSyncException if the given expression is not true.
464 * The message of the exception is formatted using the 'msg' and
465 * 'formatArgs' parameters.
466 */
467 private void checkSync(boolean expression, String msg,
468 Object... formatArgs) throws JournalOutOfSyncException {
469 if (!expression) {
470 throw new JournalOutOfSyncException(String.format(msg, formatArgs));
471 }
472 }
473
474 /**
475 * @throws AssertionError if the given expression is not true.
476 * The message of the exception is formatted using the 'msg' and
477 * 'formatArgs' parameters.
478 *
479 * This should be used in preference to Java's built-in assert in
480 * non-performance-critical paths, where a failure of this invariant
481 * might cause the protocol to lose data.
482 */
483 private void alwaysAssert(boolean expression, String msg,
484 Object... formatArgs) {
485 if (!expression) {
486 throw new AssertionError(String.format(msg, formatArgs));
487 }
488 }
489
490 /**
491 * Start a new segment at the given txid. The previous segment
492 * must have already been finalized.
493 */
494 public synchronized void startLogSegment(RequestInfo reqInfo, long txid,
495 int layoutVersion) throws IOException {
496 assert fjm != null;
497 checkFormatted();
498 checkRequest(reqInfo);
499
500 if (curSegment != null) {
501 LOG.warn("Client is requesting a new log segment " + txid +
502 " though we are already writing " + curSegment + ". " +
503 "Aborting the current segment in order to begin the new one.");
504 // The writer may have lost a connection to us and is now
505 // re-connecting after the connection came back.
506 // We should abort our own old segment.
507 abortCurSegment();
508 }
509
510 // Paranoid sanity check: we should never overwrite a finalized log file.
511 // Additionally, if it's in-progress, it should have at most 1 transaction.
512 // This can happen if the writer crashes exactly at the start of a segment.
513 EditLogFile existing = fjm.getLogFile(txid);
514 if (existing != null) {
515 if (!existing.isInProgress()) {
516 throw new IllegalStateException("Already have a finalized segment " +
517 existing + " beginning at " + txid);
518 }
519
520 // If it's in-progress, it should only contain one transaction,
521 // because the "startLogSegment" transaction is written alone at the
522 // start of each segment.
523 existing.scanLog();
524 if (existing.getLastTxId() != existing.getFirstTxId()) {
525 throw new IllegalStateException("The log file " +
526 existing + " seems to contain valid transactions");
527 }
528 }
529
530 long curLastWriterEpoch = lastWriterEpoch.get();
531 if (curLastWriterEpoch != reqInfo.getEpoch()) {
532 LOG.info("Updating lastWriterEpoch from " + curLastWriterEpoch +
533 " to " + reqInfo.getEpoch() + " for client " +
534 Server.getRemoteIp());
535 lastWriterEpoch.set(reqInfo.getEpoch());
536 }
537
538 // The fact that we are starting a segment at this txid indicates
539 // that any previous recovery for this same segment was aborted.
540 // Otherwise, no writer would have started writing. So, we can
541 // remove the record of the older segment here.
542 purgePaxosDecision(txid);
543
544 curSegment = fjm.startLogSegment(txid, layoutVersion);
545 curSegmentTxId = txid;
546 nextTxId = txid;
547 }
548
549 /**
550 * Finalize the log segment at the given transaction ID.
551 */
552 public synchronized void finalizeLogSegment(RequestInfo reqInfo, long startTxId,
553 long endTxId) throws IOException {
554 checkFormatted();
555 checkRequest(reqInfo);
556
557 boolean needsValidation = true;
558
559 // Finalizing the log that the writer was just writing.
560 if (startTxId == curSegmentTxId) {
561 if (curSegment != null) {
562 curSegment.close();
563 curSegment = null;
564 curSegmentTxId = HdfsConstants.INVALID_TXID;
565 }
566
567 checkSync(nextTxId == endTxId + 1,
568 "Trying to finalize in-progress log segment %s to end at " +
569 "txid %s but only written up to txid %s",
570 startTxId, endTxId, nextTxId - 1);
571 // No need to validate the edit log if the client is finalizing
572 // the log segment that it was just writing to.
573 needsValidation = false;
574 }
575
576 FileJournalManager.EditLogFile elf = fjm.getLogFile(startTxId);
577 if (elf == null) {
578 throw new JournalOutOfSyncException("No log file to finalize at " +
579 "transaction ID " + startTxId);
580 }
581
582 if (elf.isInProgress()) {
583 if (needsValidation) {
584 LOG.info("Validating log segment " + elf.getFile() + " about to be " +
585 "finalized");
586 elf.scanLog();
587
588 checkSync(elf.getLastTxId() == endTxId,
589 "Trying to finalize in-progress log segment %s to end at " +
590 "txid %s but log %s on disk only contains up to txid %s",
591 startTxId, endTxId, elf.getFile(), elf.getLastTxId());
592 }
593 fjm.finalizeLogSegment(startTxId, endTxId);
594 } else {
595 Preconditions.checkArgument(endTxId == elf.getLastTxId(),
596 "Trying to re-finalize already finalized log " +
597 elf + " with different endTxId " + endTxId);
598 }
599
600 // Once logs are finalized, a different length will never be decided.
601 // During recovery, we treat a finalized segment the same as an accepted
602 // recovery. Thus, we no longer need to keep track of the previously-
603 // accepted decision. The existence of the finalized log segment is enough.
604 purgePaxosDecision(elf.getFirstTxId());
605 }
606
607 /**
608 * @see JournalManager#purgeLogsOlderThan(long)
609 */
610 public synchronized void purgeLogsOlderThan(RequestInfo reqInfo,
611 long minTxIdToKeep) throws IOException {
612 checkFormatted();
613 checkRequest(reqInfo);
614
615 storage.purgeDataOlderThan(minTxIdToKeep);
616 }
617
618 /**
619 * Remove the previously-recorded 'accepted recovery' information
620 * for a given log segment, once it is no longer necessary.
621 * @param segmentTxId the transaction ID to purge
622 * @throws IOException if the file could not be deleted
623 */
624 private void purgePaxosDecision(long segmentTxId) throws IOException {
625 File paxosFile = storage.getPaxosFile(segmentTxId);
626 if (paxosFile.exists()) {
627 if (!paxosFile.delete()) {
628 throw new IOException("Unable to delete paxos file " + paxosFile);
629 }
630 }
631 }
632
633 /**
634 * @see QJournalProtocol#getEditLogManifest(String, long, boolean)
635 */
636 public RemoteEditLogManifest getEditLogManifest(long sinceTxId,
637 boolean inProgressOk) throws IOException {
638 // No need to checkRequest() here - anyone may ask for the list
639 // of segments.
640 checkFormatted();
641
642 List<RemoteEditLog> logs = fjm.getRemoteEditLogs(sinceTxId, inProgressOk);
643
644 if (inProgressOk) {
645 RemoteEditLog log = null;
646 for (Iterator<RemoteEditLog> iter = logs.iterator(); iter.hasNext();) {
647 log = iter.next();
648 if (log.isInProgress()) {
649 iter.remove();
650 break;
651 }
652 }
653 if (log != null && log.isInProgress()) {
654 logs.add(new RemoteEditLog(log.getStartTxId(), getHighestWrittenTxId()));
655 }
656 }
657
658 return new RemoteEditLogManifest(logs);
659 }
660
661 /**
662 * @return the current state of the given segment, or null if the
663 * segment does not exist.
664 */
665 @VisibleForTesting
666 SegmentStateProto getSegmentInfo(long segmentTxId)
667 throws IOException {
668 EditLogFile elf = fjm.getLogFile(segmentTxId);
669 if (elf == null) {
670 return null;
671 }
672 if (elf.isInProgress()) {
673 elf.scanLog();
674 }
675 if (elf.getLastTxId() == HdfsConstants.INVALID_TXID) {
676 LOG.info("Edit log file " + elf + " appears to be empty. " +
677 "Moving it aside...");
678 elf.moveAsideEmptyFile();
679 return null;
680 }
681 SegmentStateProto ret = SegmentStateProto.newBuilder()
682 .setStartTxId(segmentTxId)
683 .setEndTxId(elf.getLastTxId())
684 .setIsInProgress(elf.isInProgress())
685 .build();
686 LOG.info("getSegmentInfo(" + segmentTxId + "): " + elf + " -> " +
687 TextFormat.shortDebugString(ret));
688 return ret;
689 }
690
691 /**
692 * @see QJournalProtocol#prepareRecovery(RequestInfo, long)
693 */
694 public synchronized PrepareRecoveryResponseProto prepareRecovery(
695 RequestInfo reqInfo, long segmentTxId) throws IOException {
696 checkFormatted();
697 checkRequest(reqInfo);
698
699 abortCurSegment();
700
701 PrepareRecoveryResponseProto.Builder builder =
702 PrepareRecoveryResponseProto.newBuilder();
703
704 PersistedRecoveryPaxosData previouslyAccepted = getPersistedPaxosData(segmentTxId);
705 completeHalfDoneAcceptRecovery(previouslyAccepted);
706
707 SegmentStateProto segInfo = getSegmentInfo(segmentTxId);
708 boolean hasFinalizedSegment = segInfo != null && !segInfo.getIsInProgress();
709
710 if (previouslyAccepted != null && !hasFinalizedSegment) {
711 SegmentStateProto acceptedState = previouslyAccepted.getSegmentState();
712 assert acceptedState.getEndTxId() == segInfo.getEndTxId() :
713 "prev accepted: " + TextFormat.shortDebugString(previouslyAccepted)+ "\n" +
714 "on disk: " + TextFormat.shortDebugString(segInfo);
715
716 builder.setAcceptedInEpoch(previouslyAccepted.getAcceptedInEpoch())
717 .setSegmentState(previouslyAccepted.getSegmentState());
718 } else {
719 if (segInfo != null) {
720 builder.setSegmentState(segInfo);
721 }
722 }
723
724 builder.setLastWriterEpoch(lastWriterEpoch.get());
725 if (committedTxnId.get() != HdfsConstants.INVALID_TXID) {
726 builder.setLastCommittedTxId(committedTxnId.get());
727 }
728
729 PrepareRecoveryResponseProto resp = builder.build();
730 LOG.info("Prepared recovery for segment " + segmentTxId + ": " +
731 TextFormat.shortDebugString(resp));
732 return resp;
733 }
734
735 /**
736 * @see QJournalProtocol#acceptRecovery(RequestInfo, QJournalProtocolProtos.SegmentStateProto, URL)
737 */
738 public synchronized void acceptRecovery(RequestInfo reqInfo,
739 SegmentStateProto segment, URL fromUrl)
740 throws IOException {
741 checkFormatted();
742 checkRequest(reqInfo);
743
744 abortCurSegment();
745
746 long segmentTxId = segment.getStartTxId();
747
748 // Basic sanity checks that the segment is well-formed and contains
749 // at least one transaction.
750 Preconditions.checkArgument(segment.getEndTxId() > 0 &&
751 segment.getEndTxId() >= segmentTxId,
752 "bad recovery state for segment %s: %s",
753 segmentTxId, TextFormat.shortDebugString(segment));
754
755 PersistedRecoveryPaxosData oldData = getPersistedPaxosData(segmentTxId);
756 PersistedRecoveryPaxosData newData = PersistedRecoveryPaxosData.newBuilder()
757 .setAcceptedInEpoch(reqInfo.getEpoch())
758 .setSegmentState(segment)
759 .build();
760
761 // If we previously acted on acceptRecovery() from a higher-numbered writer,
762 // this call is out of sync. We should never actually trigger this, since the
763 // checkRequest() call above should filter non-increasing epoch numbers.
764 if (oldData != null) {
765 alwaysAssert(oldData.getAcceptedInEpoch() <= reqInfo.getEpoch(),
766 "Bad paxos transition, out-of-order epochs.\nOld: %s\nNew: %s\n",
767 oldData, newData);
768 }
769
770 File syncedFile = null;
771
772 SegmentStateProto currentSegment = getSegmentInfo(segmentTxId);
773 if (currentSegment == null ||
774 currentSegment.getEndTxId() != segment.getEndTxId()) {
775 if (currentSegment == null) {
776 LOG.info("Synchronizing log " + TextFormat.shortDebugString(segment) +
777 ": no current segment in place");
778
779 // Update the highest txid for lag metrics
780 highestWrittenTxId = Math.max(segment.getEndTxId(),
781 highestWrittenTxId);
782 } else {
783 LOG.info("Synchronizing log " + TextFormat.shortDebugString(segment) +
784 ": old segment " + TextFormat.shortDebugString(currentSegment) +
785 " is not the right length");
786
787 // Paranoid sanity check: if the new log is shorter than the log we
788 // currently have, we should not end up discarding any transactions
789 // which are already Committed.
790 if (txnRange(currentSegment).contains(committedTxnId.get()) &&
791 !txnRange(segment).contains(committedTxnId.get())) {
792 throw new AssertionError(
793 "Cannot replace segment " +
794 TextFormat.shortDebugString(currentSegment) +
795 " with new segment " +
796 TextFormat.shortDebugString(segment) +
797 ": would discard already-committed txn " +
798 committedTxnId.get());
799 }
800
801 // Another paranoid check: we should not be asked to synchronize a log
802 // on top of a finalized segment.
803 alwaysAssert(currentSegment.getIsInProgress(),
804 "Should never be asked to synchronize a different log on top of an " +
805 "already-finalized segment");
806
807 // If we're shortening the log, update our highest txid
808 // used for lag metrics.
809 if (txnRange(currentSegment).contains(highestWrittenTxId)) {
810 highestWrittenTxId = segment.getEndTxId();
811 }
812 }
813 syncedFile = syncLog(reqInfo, segment, fromUrl);
814
815 } else {
816 LOG.info("Skipping download of log " +
817 TextFormat.shortDebugString(segment) +
818 ": already have up-to-date logs");
819 }
820
821 // This is one of the few places in the protocol where we have a single
822 // RPC that results in two distinct actions:
823 //
824 // - 1) Downloads the new log segment data (above)
825 // - 2) Records the new Paxos data about the synchronized segment (below)
826 //
827 // These need to be treated as a transaction from the perspective
828 // of any external process. We do this by treating the persistPaxosData()
829 // success as the "commit" of an atomic transaction. If we fail before
830 // this point, the downloaded edit log will only exist at a temporary
831 // path, and thus not change any externally visible state. If we fail
832 // after this point, then any future prepareRecovery() call will see
833 // the Paxos data, and by calling completeHalfDoneAcceptRecovery() will
834 // roll forward the rename of the referenced log file.
835 //
836 // See also: HDFS-3955
837 //
838 // The fault points here are exercised by the randomized fault injection
839 // test case to ensure that this atomic "transaction" operates correctly.
840 JournalFaultInjector.get().beforePersistPaxosData();
841 persistPaxosData(segmentTxId, newData);
842 JournalFaultInjector.get().afterPersistPaxosData();
843
844 if (syncedFile != null) {
845 FileUtil.replaceFile(syncedFile,
846 storage.getInProgressEditLog(segmentTxId));
847 }
848
849 LOG.info("Accepted recovery for segment " + segmentTxId + ": " +
850 TextFormat.shortDebugString(newData));
851 }
852
853 private Range<Long> txnRange(SegmentStateProto seg) {
854 Preconditions.checkArgument(seg.hasEndTxId(),
855 "invalid segment: %s", seg);
856 return Ranges.closed(seg.getStartTxId(), seg.getEndTxId());
857 }
858
859 /**
860 * Synchronize a log segment from another JournalNode. The log is
861 * downloaded from the provided URL into a temporary location on disk,
862 * which is named based on the current request's epoch.
863 *
864 * @return the temporary location of the downloaded file
865 */
866 private File syncLog(RequestInfo reqInfo,
867 final SegmentStateProto segment, final URL url) throws IOException {
868 final File tmpFile = storage.getSyncLogTemporaryFile(
869 segment.getStartTxId(), reqInfo.getEpoch());
870 final List<File> localPaths = ImmutableList.of(tmpFile);
871
872 LOG.info("Synchronizing log " +
873 TextFormat.shortDebugString(segment) + " from " + url);
874 SecurityUtil.doAsLoginUser(
875 new PrivilegedExceptionAction<Void>() {
876 @Override
877 public Void run() throws IOException {
878 // We may have lost our ticket since last checkpoint, log in again, just in case
879 if (UserGroupInformation.isSecurityEnabled()) {
880 UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
881 }
882
883 boolean success = false;
884 try {
885 TransferFsImage.doGetUrl(url, localPaths, storage, true);
886 assert tmpFile.exists();
887 success = true;
888 } finally {
889 if (!success) {
890 if (!tmpFile.delete()) {
891 LOG.warn("Failed to delete temporary file " + tmpFile);
892 }
893 }
894 }
895 return null;
896 }
897 });
898 return tmpFile;
899 }
900
901
902 /**
903 * In the case the node crashes in between downloading a log segment
904 * and persisting the associated paxos recovery data, the log segment
905 * will be left in its temporary location on disk. Given the paxos data,
906 * we can check if this was indeed the case, and "roll forward"
907 * the atomic operation.
908 *
909 * See the inline comments in
910 * {@link #acceptRecovery(RequestInfo, SegmentStateProto, URL)} for more
911 * details.
912 *
913 * @throws IOException if the temporary file is unable to be renamed into
914 * place
915 */
916 private void completeHalfDoneAcceptRecovery(
917 PersistedRecoveryPaxosData paxosData) throws IOException {
918 if (paxosData == null) {
919 return;
920 }
921
922 long segmentId = paxosData.getSegmentState().getStartTxId();
923 long epoch = paxosData.getAcceptedInEpoch();
924
925 File tmp = storage.getSyncLogTemporaryFile(segmentId, epoch);
926
927 if (tmp.exists()) {
928 File dst = storage.getInProgressEditLog(segmentId);
929 LOG.info("Rolling forward previously half-completed synchronization: " +
930 tmp + " -> " + dst);
931 FileUtil.replaceFile(tmp, dst);
932 }
933 }
934
935 /**
936 * Retrieve the persisted data for recovering the given segment from disk.
937 */
938 private PersistedRecoveryPaxosData getPersistedPaxosData(long segmentTxId)
939 throws IOException {
940 File f = storage.getPaxosFile(segmentTxId);
941 if (!f.exists()) {
942 // Default instance has no fields filled in (they're optional)
943 return null;
944 }
945
946 InputStream in = new FileInputStream(f);
947 try {
948 PersistedRecoveryPaxosData ret = PersistedRecoveryPaxosData.parseDelimitedFrom(in);
949 Preconditions.checkState(ret != null &&
950 ret.getSegmentState().getStartTxId() == segmentTxId,
951 "Bad persisted data for segment %s: %s",
952 segmentTxId, ret);
953 return ret;
954 } finally {
955 IOUtils.closeStream(in);
956 }
957 }
958
959 /**
960 * Persist data for recovering the given segment from disk.
961 */
962 private void persistPaxosData(long segmentTxId,
963 PersistedRecoveryPaxosData newData) throws IOException {
964 File f = storage.getPaxosFile(segmentTxId);
965 boolean success = false;
966 AtomicFileOutputStream fos = new AtomicFileOutputStream(f);
967 try {
968 newData.writeDelimitedTo(fos);
969 fos.write('\n');
970 // Write human-readable data after the protobuf. This is only
971 // to assist in debugging -- it's not parsed at all.
972 OutputStreamWriter writer = new OutputStreamWriter(fos, Charsets.UTF_8);
973
974 writer.write(String.valueOf(newData));
975 writer.write('\n');
976 writer.flush();
977
978 fos.flush();
979 success = true;
980 } finally {
981 if (success) {
982 IOUtils.closeStream(fos);
983 } else {
984 fos.abort();
985 }
986 }
987 }
988
989 synchronized void discardSegments(long startTxId) throws IOException {
990 storage.getJournalManager().discardSegments(startTxId);
991 // we delete all the segments after the startTxId. let's reset committedTxnId
992 committedTxnId.set(startTxId - 1);
993 }
994
995 public synchronized void doPreUpgrade() throws IOException {
996 storage.getJournalManager().doPreUpgrade();
997 }
998
999 public synchronized void doUpgrade(StorageInfo sInfo) throws IOException {
1000 long oldCTime = storage.getCTime();
1001 storage.cTime = sInfo.cTime;
1002 int oldLV = storage.getLayoutVersion();
1003 storage.layoutVersion = sInfo.layoutVersion;
1004 LOG.info("Starting upgrade of edits directory: "
1005 + ".\n old LV = " + oldLV
1006 + "; old CTime = " + oldCTime
1007 + ".\n new LV = " + storage.getLayoutVersion()
1008 + "; new CTime = " + storage.getCTime());
1009 storage.getJournalManager().doUpgrade(storage);
1010 storage.createPaxosDir();
1011
1012 // Copy over the contents of the epoch data files to the new dir.
1013 File currentDir = storage.getSingularStorageDir().getCurrentDir();
1014 File previousDir = storage.getSingularStorageDir().getPreviousDir();
1015
1016 PersistentLongFile prevLastPromisedEpoch = new PersistentLongFile(
1017 new File(previousDir, LAST_PROMISED_FILENAME), 0);
1018 PersistentLongFile prevLastWriterEpoch = new PersistentLongFile(
1019 new File(previousDir, LAST_WRITER_EPOCH), 0);
1020
1021 lastPromisedEpoch = new PersistentLongFile(
1022 new File(currentDir, LAST_PROMISED_FILENAME), 0);
1023 lastWriterEpoch = new PersistentLongFile(
1024 new File(currentDir, LAST_WRITER_EPOCH), 0);
1025
1026 lastPromisedEpoch.set(prevLastPromisedEpoch.get());
1027 lastWriterEpoch.set(prevLastWriterEpoch.get());
1028 }
1029
1030 public synchronized void doFinalize() throws IOException {
1031 LOG.info("Finalizing upgrade for journal "
1032 + storage.getRoot() + "."
1033 + (storage.getLayoutVersion()==0 ? "" :
1034 "\n cur LV = " + storage.getLayoutVersion()
1035 + "; cur CTime = " + storage.getCTime()));
1036 storage.getJournalManager().doFinalize();
1037 }
1038
1039 public Boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
1040 int targetLayoutVersion) throws IOException {
1041 return this.storage.getJournalManager().canRollBack(storage, prevStorage,
1042 targetLayoutVersion);
1043 }
1044
1045 public void doRollback() throws IOException {
1046 storage.getJournalManager().doRollback();
1047 }
1048
1049 public Long getJournalCTime() throws IOException {
1050 return storage.getJournalManager().getJournalCTime();
1051 }
1052 }