瀏覽代碼

Fix JavaDoc in EditLogOutputStream given that it doesn't inherit from OutputStream

Part of response to Konstantin's review.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1073@1150888 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 年之前
父節點
當前提交
1fec2bb0b2

+ 0 - 5
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java

@@ -97,11 +97,6 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
     return JournalType.BACKUP;
   }
 
-  @Override // EditLogOutputStream
-  public void write(int b) throws IOException {
-    throw new IOException("Not implemented");
-  }
-
   @Override
   void write(byte[] data, int i, int length) throws IOException {
     throw new IOException("Not implemented");

+ 5 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java

@@ -90,9 +90,11 @@ class EditLogFileOutputStream extends EditLogOutputStream {
     return JournalType.FILE;
   }
 
-  /** {@inheritDoc} */
-  @Override
-  public void write(int b) throws IOException {
+  /**
+   * Write a single byte to the output stream.
+   * @param b the byte to write
+   */
+  private void write(int b) throws IOException {
     if (fp == null) {
       throw new IOException("Trying to use aborted output stream");
     }

+ 6 - 6
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java

@@ -29,8 +29,7 @@ import org.apache.hadoop.io.Writable;
  * A generic abstract class to support journaling of edits logs into 
  * a persistent storage.
  */
-abstract class EditLogOutputStream 
-implements JournalStream {
+abstract class EditLogOutputStream implements JournalStream {
   // these are statistics counters
   private long numSync;        // number of sync(s) to disk
   private long totalTimeSync;  // total time to sync
@@ -39,9 +38,6 @@ implements JournalStream {
     numSync = totalTimeSync = 0;
   }
 
-  /** {@inheritDoc} */
-  abstract public void write(int b) throws IOException;
-
   /**
    * Write edits log record into the stream.
    * The record is represented by operation name and
@@ -70,7 +66,11 @@ implements JournalStream {
    */
   abstract void create() throws IOException;
 
-  /** {@inheritDoc} */
+  /**
+   * Close the journal.
+   * @throws IOException if the journal can't be closed,
+   *         or if there are unflushed edits
+   */
   abstract public void close() throws IOException;
 
   /**

+ 5 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java

@@ -45,12 +45,15 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.io.Writable;
+
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 import com.google.common.collect.ImmutableSet;
 /**
@@ -128,7 +131,8 @@ public class TestStorageRestore {
           EditLogOutputStream mockStream = spy(j.getCurrentStream());
           j.setCurrentStreamForTests(mockStream);
           doThrow(new IOException("Injected fault: write")).
-            when(mockStream).write(anyByte());
+            when(mockStream).write(Mockito.anyByte(),
+                Mockito.anyLong(), (Writable[]) Mockito.anyVararg());
         }
       }
     }