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.server.namenode;
019
020 import static org.apache.hadoop.util.ExitUtil.terminate;
021
022 import java.io.IOException;
023 import java.util.ArrayList;
024 import java.util.Collection;
025 import java.util.Collections;
026 import java.util.Comparator;
027 import java.util.LinkedList;
028 import java.util.List;
029 import java.util.PriorityQueue;
030 import java.util.SortedSet;
031 import java.util.concurrent.CopyOnWriteArrayList;
032
033 import org.apache.commons.logging.Log;
034 import org.apache.commons.logging.LogFactory;
035 import org.apache.hadoop.classification.InterfaceAudience;
036 import org.apache.hadoop.hdfs.server.common.Storage;
037 import org.apache.hadoop.hdfs.server.common.StorageInfo;
038 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
039 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
040 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
041
042 import static org.apache.hadoop.util.ExitUtil.terminate;
043
044 import com.google.common.base.Preconditions;
045 import com.google.common.collect.ComparisonChain;
046 import com.google.common.collect.ImmutableList;
047 import com.google.common.collect.ImmutableListMultimap;
048 import com.google.common.collect.Lists;
049 import com.google.common.collect.Multimaps;
050 import com.google.common.collect.Sets;
051
052 /**
053 * Manages a collection of Journals. None of the methods are synchronized, it is
054 * assumed that FSEditLog methods, that use this class, use proper
055 * synchronization.
056 */
057 public class JournalSet implements JournalManager {
058
059 static final Log LOG = LogFactory.getLog(FSEditLog.class);
060
061 static final public Comparator<EditLogInputStream>
062 EDIT_LOG_INPUT_STREAM_COMPARATOR = new Comparator<EditLogInputStream>() {
063 @Override
064 public int compare(EditLogInputStream a, EditLogInputStream b) {
065 return ComparisonChain.start().
066 compare(a.getFirstTxId(), b.getFirstTxId()).
067 compare(b.getLastTxId(), a.getLastTxId()).
068 result();
069 }
070 };
071
072 /**
073 * Container for a JournalManager paired with its currently
074 * active stream.
075 *
076 * If a Journal gets disabled due to an error writing to its
077 * stream, then the stream will be aborted and set to null.
078 */
079 static class JournalAndStream implements CheckableNameNodeResource {
080 private final JournalManager journal;
081 private boolean disabled = false;
082 private EditLogOutputStream stream;
083 private final boolean required;
084 private final boolean shared;
085
086 public JournalAndStream(JournalManager manager, boolean required,
087 boolean shared) {
088 this.journal = manager;
089 this.required = required;
090 this.shared = shared;
091 }
092
093 public void startLogSegment(long txId, int layoutVersion) throws IOException {
094 Preconditions.checkState(stream == null);
095 disabled = false;
096 stream = journal.startLogSegment(txId, layoutVersion);
097 }
098
099 /**
100 * Closes the stream, also sets it to null.
101 */
102 public void closeStream() throws IOException {
103 if (stream == null) return;
104 stream.close();
105 stream = null;
106 }
107
108 /**
109 * Close the Journal and Stream
110 */
111 public void close() throws IOException {
112 closeStream();
113
114 journal.close();
115 }
116
117 /**
118 * Aborts the stream, also sets it to null.
119 */
120 public void abort() {
121 if (stream == null) return;
122 try {
123 stream.abort();
124 } catch (IOException ioe) {
125 LOG.error("Unable to abort stream " + stream, ioe);
126 }
127 stream = null;
128 }
129
130 boolean isActive() {
131 return stream != null;
132 }
133
134 /**
135 * Should be used outside JournalSet only for testing.
136 */
137 EditLogOutputStream getCurrentStream() {
138 return stream;
139 }
140
141 @Override
142 public String toString() {
143 return "JournalAndStream(mgr=" + journal +
144 ", " + "stream=" + stream + ")";
145 }
146
147 void setCurrentStreamForTests(EditLogOutputStream stream) {
148 this.stream = stream;
149 }
150
151 JournalManager getManager() {
152 return journal;
153 }
154
155 boolean isDisabled() {
156 return disabled;
157 }
158
159 private void setDisabled(boolean disabled) {
160 this.disabled = disabled;
161 }
162
163 @Override
164 public boolean isResourceAvailable() {
165 return !isDisabled();
166 }
167
168 @Override
169 public boolean isRequired() {
170 return required;
171 }
172
173 public boolean isShared() {
174 return shared;
175 }
176 }
177
178 // COW implementation is necessary since some users (eg the web ui) call
179 // getAllJournalStreams() and then iterate. Since this is rarely
180 // mutated, there is no performance concern.
181 private final List<JournalAndStream> journals =
182 new CopyOnWriteArrayList<JournalSet.JournalAndStream>();
183 final int minimumRedundantJournals;
184
185 JournalSet(int minimumRedundantResources) {
186 this.minimumRedundantJournals = minimumRedundantResources;
187 }
188
189 @Override
190 public void format(NamespaceInfo nsInfo) throws IOException {
191 // The operation is done by FSEditLog itself
192 throw new UnsupportedOperationException();
193 }
194
195 @Override
196 public boolean hasSomeData() throws IOException {
197 // This is called individually on the underlying journals,
198 // not on the JournalSet.
199 throw new UnsupportedOperationException();
200 }
201
202
203 @Override
204 public EditLogOutputStream startLogSegment(final long txId,
205 final int layoutVersion) throws IOException {
206 mapJournalsAndReportErrors(new JournalClosure() {
207 @Override
208 public void apply(JournalAndStream jas) throws IOException {
209 jas.startLogSegment(txId, layoutVersion);
210 }
211 }, "starting log segment " + txId);
212 return new JournalSetOutputStream();
213 }
214
215 @Override
216 public void finalizeLogSegment(final long firstTxId, final long lastTxId)
217 throws IOException {
218 mapJournalsAndReportErrors(new JournalClosure() {
219 @Override
220 public void apply(JournalAndStream jas) throws IOException {
221 if (jas.isActive()) {
222 jas.closeStream();
223 jas.getManager().finalizeLogSegment(firstTxId, lastTxId);
224 }
225 }
226 }, "finalize log segment " + firstTxId + ", " + lastTxId);
227 }
228
229 @Override
230 public void close() throws IOException {
231 mapJournalsAndReportErrors(new JournalClosure() {
232 @Override
233 public void apply(JournalAndStream jas) throws IOException {
234 jas.close();
235 }
236 }, "close journal");
237 }
238
239 /**
240 * In this function, we get a bunch of streams from all of our JournalManager
241 * objects. Then we add these to the collection one by one.
242 *
243 * @param streams The collection to add the streams to. It may or
244 * may not be sorted-- this is up to the caller.
245 * @param fromTxId The transaction ID to start looking for streams at
246 * @param inProgressOk Should we consider unfinalized streams?
247 */
248 @Override
249 public void selectInputStreams(Collection<EditLogInputStream> streams,
250 long fromTxId, boolean inProgressOk) throws IOException {
251 final PriorityQueue<EditLogInputStream> allStreams =
252 new PriorityQueue<EditLogInputStream>(64,
253 EDIT_LOG_INPUT_STREAM_COMPARATOR);
254 for (JournalAndStream jas : journals) {
255 if (jas.isDisabled()) {
256 LOG.info("Skipping jas " + jas + " since it's disabled");
257 continue;
258 }
259 try {
260 jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk);
261 } catch (IOException ioe) {
262 LOG.warn("Unable to determine input streams from " + jas.getManager() +
263 ". Skipping.", ioe);
264 }
265 }
266 chainAndMakeRedundantStreams(streams, allStreams, fromTxId);
267 }
268
269 public static void chainAndMakeRedundantStreams(
270 Collection<EditLogInputStream> outStreams,
271 PriorityQueue<EditLogInputStream> allStreams, long fromTxId) {
272 // We want to group together all the streams that start on the same start
273 // transaction ID. To do this, we maintain an accumulator (acc) of all
274 // the streams we've seen at a given start transaction ID. When we see a
275 // higher start transaction ID, we select a stream from the accumulator and
276 // clear it. Then we begin accumulating streams with the new, higher start
277 // transaction ID.
278 LinkedList<EditLogInputStream> acc =
279 new LinkedList<EditLogInputStream>();
280 EditLogInputStream elis;
281 while ((elis = allStreams.poll()) != null) {
282 if (acc.isEmpty()) {
283 acc.add(elis);
284 } else {
285 long accFirstTxId = acc.get(0).getFirstTxId();
286 if (accFirstTxId == elis.getFirstTxId()) {
287 acc.add(elis);
288 } else if (accFirstTxId < elis.getFirstTxId()) {
289 outStreams.add(new RedundantEditLogInputStream(acc, fromTxId));
290 acc.clear();
291 acc.add(elis);
292 } else if (accFirstTxId > elis.getFirstTxId()) {
293 throw new RuntimeException("sorted set invariants violated! " +
294 "Got stream with first txid " + elis.getFirstTxId() +
295 ", but the last firstTxId was " + accFirstTxId);
296 }
297 }
298 }
299 if (!acc.isEmpty()) {
300 outStreams.add(new RedundantEditLogInputStream(acc, fromTxId));
301 acc.clear();
302 }
303 }
304
305 /**
306 * Returns true if there are no journals, all redundant journals are disabled,
307 * or any required journals are disabled.
308 *
309 * @return True if there no journals, all redundant journals are disabled,
310 * or any required journals are disabled.
311 */
312 public boolean isEmpty() {
313 return !NameNodeResourcePolicy.areResourcesAvailable(journals,
314 minimumRedundantJournals);
315 }
316
317 /**
318 * Called when some journals experience an error in some operation.
319 */
320 private void disableAndReportErrorOnJournals(List<JournalAndStream> badJournals) {
321 if (badJournals == null || badJournals.isEmpty()) {
322 return; // nothing to do
323 }
324
325 for (JournalAndStream j : badJournals) {
326 LOG.error("Disabling journal " + j);
327 j.abort();
328 j.setDisabled(true);
329 }
330 }
331
332 /**
333 * Implementations of this interface encapsulate operations that can be
334 * iteratively applied on all the journals. For example see
335 * {@link JournalSet#mapJournalsAndReportErrors}.
336 */
337 private interface JournalClosure {
338 /**
339 * The operation on JournalAndStream.
340 * @param jas Object on which operations are performed.
341 * @throws IOException
342 */
343 public void apply(JournalAndStream jas) throws IOException;
344 }
345
346 /**
347 * Apply the given operation across all of the journal managers, disabling
348 * any for which the closure throws an IOException.
349 * @param closure {@link JournalClosure} object encapsulating the operation.
350 * @param status message used for logging errors (e.g. "opening journal")
351 * @throws IOException If the operation fails on all the journals.
352 */
353 private void mapJournalsAndReportErrors(
354 JournalClosure closure, String status) throws IOException{
355
356 List<JournalAndStream> badJAS = Lists.newLinkedList();
357 for (JournalAndStream jas : journals) {
358 try {
359 closure.apply(jas);
360 } catch (Throwable t) {
361 if (jas.isRequired()) {
362 final String msg = "Error: " + status + " failed for required journal ("
363 + jas + ")";
364 LOG.fatal(msg, t);
365 // If we fail on *any* of the required journals, then we must not
366 // continue on any of the other journals. Abort them to ensure that
367 // retry behavior doesn't allow them to keep going in any way.
368 abortAllJournals();
369 // the current policy is to shutdown the NN on errors to shared edits
370 // dir. There are many code paths to shared edits failures - syncs,
371 // roll of edits etc. All of them go through this common function
372 // where the isRequired() check is made. Applying exit policy here
373 // to catch all code paths.
374 terminate(1, msg);
375 } else {
376 LOG.error("Error: " + status + " failed for (journal " + jas + ")", t);
377 badJAS.add(jas);
378 }
379 }
380 }
381 disableAndReportErrorOnJournals(badJAS);
382 if (!NameNodeResourcePolicy.areResourcesAvailable(journals,
383 minimumRedundantJournals)) {
384 String message = status + " failed for too many journals";
385 LOG.error("Error: " + message);
386 throw new IOException(message);
387 }
388 }
389
390 /**
391 * Abort all of the underlying streams.
392 */
393 private void abortAllJournals() {
394 for (JournalAndStream jas : journals) {
395 if (jas.isActive()) {
396 jas.abort();
397 }
398 }
399 }
400
401 /**
402 * An implementation of EditLogOutputStream that applies a requested method on
403 * all the journals that are currently active.
404 */
405 private class JournalSetOutputStream extends EditLogOutputStream {
406
407 JournalSetOutputStream() throws IOException {
408 super();
409 }
410
411 @Override
412 public void write(final FSEditLogOp op)
413 throws IOException {
414 mapJournalsAndReportErrors(new JournalClosure() {
415 @Override
416 public void apply(JournalAndStream jas) throws IOException {
417 if (jas.isActive()) {
418 jas.getCurrentStream().write(op);
419 }
420 }
421 }, "write op");
422 }
423
424 @Override
425 public void writeRaw(final byte[] data, final int offset, final int length)
426 throws IOException {
427 mapJournalsAndReportErrors(new JournalClosure() {
428 @Override
429 public void apply(JournalAndStream jas) throws IOException {
430 if (jas.isActive()) {
431 jas.getCurrentStream().writeRaw(data, offset, length);
432 }
433 }
434 }, "write bytes");
435 }
436
437 @Override
438 public void create(final int layoutVersion) throws IOException {
439 mapJournalsAndReportErrors(new JournalClosure() {
440 @Override
441 public void apply(JournalAndStream jas) throws IOException {
442 if (jas.isActive()) {
443 jas.getCurrentStream().create(layoutVersion);
444 }
445 }
446 }, "create");
447 }
448
449 @Override
450 public void close() throws IOException {
451 mapJournalsAndReportErrors(new JournalClosure() {
452 @Override
453 public void apply(JournalAndStream jas) throws IOException {
454 jas.closeStream();
455 }
456 }, "close");
457 }
458
459 @Override
460 public void abort() throws IOException {
461 mapJournalsAndReportErrors(new JournalClosure() {
462 @Override
463 public void apply(JournalAndStream jas) throws IOException {
464 jas.abort();
465 }
466 }, "abort");
467 }
468
469 @Override
470 public void setReadyToFlush() throws IOException {
471 mapJournalsAndReportErrors(new JournalClosure() {
472 @Override
473 public void apply(JournalAndStream jas) throws IOException {
474 if (jas.isActive()) {
475 jas.getCurrentStream().setReadyToFlush();
476 }
477 }
478 }, "setReadyToFlush");
479 }
480
481 @Override
482 protected void flushAndSync(final boolean durable) throws IOException {
483 mapJournalsAndReportErrors(new JournalClosure() {
484 @Override
485 public void apply(JournalAndStream jas) throws IOException {
486 if (jas.isActive()) {
487 jas.getCurrentStream().flushAndSync(durable);
488 }
489 }
490 }, "flushAndSync");
491 }
492
493 @Override
494 public void flush() throws IOException {
495 mapJournalsAndReportErrors(new JournalClosure() {
496 @Override
497 public void apply(JournalAndStream jas) throws IOException {
498 if (jas.isActive()) {
499 jas.getCurrentStream().flush();
500 }
501 }
502 }, "flush");
503 }
504
505 @Override
506 public boolean shouldForceSync() {
507 for (JournalAndStream js : journals) {
508 if (js.isActive() && js.getCurrentStream().shouldForceSync()) {
509 return true;
510 }
511 }
512 return false;
513 }
514
515 @Override
516 protected long getNumSync() {
517 for (JournalAndStream jas : journals) {
518 if (jas.isActive()) {
519 return jas.getCurrentStream().getNumSync();
520 }
521 }
522 return 0;
523 }
524 }
525
526 @Override
527 public void setOutputBufferCapacity(final int size) {
528 try {
529 mapJournalsAndReportErrors(new JournalClosure() {
530 @Override
531 public void apply(JournalAndStream jas) throws IOException {
532 jas.getManager().setOutputBufferCapacity(size);
533 }
534 }, "setOutputBufferCapacity");
535 } catch (IOException e) {
536 LOG.error("Error in setting outputbuffer capacity");
537 }
538 }
539
540 List<JournalAndStream> getAllJournalStreams() {
541 return journals;
542 }
543
544 List<JournalManager> getJournalManagers() {
545 List<JournalManager> jList = new ArrayList<JournalManager>();
546 for (JournalAndStream j : journals) {
547 jList.add(j.getManager());
548 }
549 return jList;
550 }
551
552 void add(JournalManager j, boolean required) {
553 add(j, required, false);
554 }
555
556 void add(JournalManager j, boolean required, boolean shared) {
557 JournalAndStream jas = new JournalAndStream(j, required, shared);
558 journals.add(jas);
559 }
560
561 void remove(JournalManager j) {
562 JournalAndStream jasToRemove = null;
563 for (JournalAndStream jas: journals) {
564 if (jas.getManager().equals(j)) {
565 jasToRemove = jas;
566 break;
567 }
568 }
569 if (jasToRemove != null) {
570 jasToRemove.abort();
571 journals.remove(jasToRemove);
572 }
573 }
574
575 @Override
576 public void purgeLogsOlderThan(final long minTxIdToKeep) throws IOException {
577 mapJournalsAndReportErrors(new JournalClosure() {
578 @Override
579 public void apply(JournalAndStream jas) throws IOException {
580 jas.getManager().purgeLogsOlderThan(minTxIdToKeep);
581 }
582 }, "purgeLogsOlderThan " + minTxIdToKeep);
583 }
584
585 @Override
586 public void recoverUnfinalizedSegments() throws IOException {
587 mapJournalsAndReportErrors(new JournalClosure() {
588 @Override
589 public void apply(JournalAndStream jas) throws IOException {
590 jas.getManager().recoverUnfinalizedSegments();
591 }
592 }, "recoverUnfinalizedSegments");
593 }
594
595 /**
596 * Return a manifest of what finalized edit logs are available. All available
597 * edit logs are returned starting from the transaction id passed. If
598 * 'fromTxId' falls in the middle of a log, that log is returned as well.
599 *
600 * @param fromTxId Starting transaction id to read the logs.
601 * @return RemoteEditLogManifest object.
602 */
603 public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId) {
604 // Collect RemoteEditLogs available from each FileJournalManager
605 List<RemoteEditLog> allLogs = Lists.newArrayList();
606 for (JournalAndStream j : journals) {
607 if (j.getManager() instanceof FileJournalManager) {
608 FileJournalManager fjm = (FileJournalManager)j.getManager();
609 try {
610 allLogs.addAll(fjm.getRemoteEditLogs(fromTxId, false));
611 } catch (Throwable t) {
612 LOG.warn("Cannot list edit logs in " + fjm, t);
613 }
614 }
615 }
616
617 // Group logs by their starting txid
618 ImmutableListMultimap<Long, RemoteEditLog> logsByStartTxId =
619 Multimaps.index(allLogs, RemoteEditLog.GET_START_TXID);
620 long curStartTxId = fromTxId;
621
622 List<RemoteEditLog> logs = Lists.newArrayList();
623 while (true) {
624 ImmutableList<RemoteEditLog> logGroup = logsByStartTxId.get(curStartTxId);
625 if (logGroup.isEmpty()) {
626 // we have a gap in logs - for example because we recovered some old
627 // storage directory with ancient logs. Clear out any logs we've
628 // accumulated so far, and then skip to the next segment of logs
629 // after the gap.
630 SortedSet<Long> startTxIds = Sets.newTreeSet(logsByStartTxId.keySet());
631 startTxIds = startTxIds.tailSet(curStartTxId);
632 if (startTxIds.isEmpty()) {
633 break;
634 } else {
635 if (LOG.isDebugEnabled()) {
636 LOG.debug("Found gap in logs at " + curStartTxId + ": " +
637 "not returning previous logs in manifest.");
638 }
639 logs.clear();
640 curStartTxId = startTxIds.first();
641 continue;
642 }
643 }
644
645 // Find the one that extends the farthest forward
646 RemoteEditLog bestLog = Collections.max(logGroup);
647 logs.add(bestLog);
648 // And then start looking from after that point
649 curStartTxId = bestLog.getEndTxId() + 1;
650 }
651 RemoteEditLogManifest ret = new RemoteEditLogManifest(logs);
652
653 if (LOG.isDebugEnabled()) {
654 LOG.debug("Generated manifest for logs since " + fromTxId + ":"
655 + ret);
656 }
657 return ret;
658 }
659
660 /**
661 * Add sync times to the buffer.
662 */
663 String getSyncTimes() {
664 StringBuilder buf = new StringBuilder();
665 for (JournalAndStream jas : journals) {
666 if (jas.isActive()) {
667 buf.append(jas.getCurrentStream().getTotalSyncTime());
668 buf.append(" ");
669 }
670 }
671 return buf.toString();
672 }
673
674 @Override
675 public void discardSegments(long startTxId) throws IOException {
676 // This operation is handled by FSEditLog directly.
677 throw new UnsupportedOperationException();
678 }
679
680 @Override
681 public void doPreUpgrade() throws IOException {
682 // This operation is handled by FSEditLog directly.
683 throw new UnsupportedOperationException();
684 }
685
686 @Override
687 public void doUpgrade(Storage storage) throws IOException {
688 // This operation is handled by FSEditLog directly.
689 throw new UnsupportedOperationException();
690 }
691
692 @Override
693 public void doFinalize() throws IOException {
694 // This operation is handled by FSEditLog directly.
695 throw new UnsupportedOperationException();
696 }
697
698 @Override
699 public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage, int targetLayoutVersion) throws IOException {
700 // This operation is handled by FSEditLog directly.
701 throw new UnsupportedOperationException();
702 }
703
704 @Override
705 public void doRollback() throws IOException {
706 // This operation is handled by FSEditLog directly.
707 throw new UnsupportedOperationException();
708 }
709
710 @Override
711 public long getJournalCTime() throws IOException {
712 // This operation is handled by FSEditLog directly.
713 throw new UnsupportedOperationException();
714 }
715 }