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 &quot;roll forward&quot;
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    }