You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2012/04/14 01:50:22 UTC

svn commit: r1326020 - in /hadoop/common/branches/HDFS-3092/hadoop-hdfs-project: ./ hadoop-hdfs/ hadoop-hdfs/src/main/java/ hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/ hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/...

Author: szetszwo
Date: Fri Apr 13 23:50:21 2012
New Revision: 1326020

URL: http://svn.apache.org/viewvc?rev=1326020&view=rev
Log:
Merge r1326009 through r1326019 from trunk.

Modified:
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/native/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java

Propchange: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Fri Apr 13 23:50:21 2012
@@ -1 +1,4 @@
 target
+.classpath
+.project
+.settings

Propchange: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs:r1326009-1326019

Modified: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1326020&r1=1326019&r2=1326020&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Fri Apr 13 23:50:21 2012
@@ -62,6 +62,9 @@ Trunk (unreleased changes)
     HDFS-3178. Add states and state handler for journal synchronization in
     JournalService.  (szetszwo)
 
+    HDFS-3273. Refactor BackupImage and FSEditLog, and rename
+    JournalListener.rollLogs(..) to startLogSegment(..).  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-2834. Add a ByteBuffer-based read API to DFSInputStream.
@@ -91,8 +94,6 @@ Trunk (unreleased changes)
     HDFS-2373. Commands using WebHDFS and hftp print unnecessary debug 
     info on the console with security enabled. (Arpit Gupta via suresh)
 
-    HDFS-2765. TestNameEditsConfigs is incorrectly swallowing IOE. (atm)
-
     HDFS-2776. Missing interface annotation on JournalSet. 
     (Brandon Li via jitendra)
 
@@ -511,6 +512,8 @@ Release 2.0.0 - UNRELEASED 
 
     HDFS-2799. Trim fs.checkpoint.dir values. (Amith D K via eli)
 
+    HDFS-2765. TestNameEditsConfigs is incorrectly swallowing IOE. (atm)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)

Propchange: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java:r1326009-1326019

Modified: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java?rev=1326020&r1=1326019&r2=1326020&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java Fri Apr 13 23:50:21 2012
@@ -60,5 +60,5 @@ public interface JournalListener {
    * Any IOException thrown from the listener is thrown back in 
    * {@link JournalProtocol#startLogSegment}
    */
-  public void rollLogs(JournalService service, long txid) throws IOException;
+  public void startLogSegment(JournalService service, long txid) throws IOException;
 }
\ No newline at end of file

Modified: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java?rev=1326020&r1=1326019&r2=1326020&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java Fri Apr 13 23:50:21 2012
@@ -329,7 +329,7 @@ public class JournalService implements J
     }
     stateHandler.isStartLogSegmentAllowed();
     verify(epoch, journalInfo);
-    listener.rollLogs(this, txid);
+    listener.startLogSegment(this, txid);
     stateHandler.startLogSegment();
   }
   

Modified: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java?rev=1326020&r1=1326019&r2=1326020&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java Fri Apr 13 23:50:21 2012
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -183,21 +184,9 @@ public class BackupImage extends FSImage
     }
     
     // write to BN's local edit log.
-    logEditsLocally(firstTxId, numTxns, data);
+    editLog.journal(firstTxId, numTxns, data);
   }
 
-  /**
-   * Write the batch of edits to the local copy of the edit logs.
-   */
-  private void logEditsLocally(long firstTxId, int numTxns, byte[] data) {
-    long expectedTxId = editLog.getLastWrittenTxId() + 1;
-    Preconditions.checkState(firstTxId == expectedTxId,
-        "received txid batch starting at %s but expected txn %s",
-        firstTxId, expectedTxId);
-    editLog.setNextTxId(firstTxId + numTxns - 1);
-    editLog.logEdit(data.length, data);
-    editLog.logSync();
-  }
 
   /**
    * Apply the batch of edits to the local namespace.
@@ -342,28 +331,9 @@ public class BackupImage extends FSImage
    * This causes the BN to also start the new edit log in its local
    * directories.
    */
-  synchronized void namenodeStartedLogSegment(long txid)
-      throws IOException {
-    LOG.info("NameNode started a new log segment at txid " + txid);
-    if (editLog.isSegmentOpen()) {
-      if (editLog.getLastWrittenTxId() == txid - 1) {
-        // We are in sync with the NN, so end and finalize the current segment
-        editLog.endCurrentLogSegment(false);
-      } else {
-        // We appear to have missed some transactions -- the NN probably
-        // lost contact with us temporarily. So, mark the current segment
-        // as aborted.
-        LOG.warn("NN started new log segment at txid " + txid +
-            ", but BN had only written up to txid " +
-            editLog.getLastWrittenTxId() +
-            "in the log segment starting at " + 
-        		editLog.getCurSegmentTxId() + ". Aborting this " +
-        		"log segment.");
-        editLog.abortCurrentLogSegment();
-      }
-    }
-    editLog.setNextTxId(txid);
-    editLog.startLogSegment(txid, false);
+  synchronized void namenodeStartedLogSegment(long txid) throws IOException {
+    editLog.startLogSegment(txid, true);
+
     if (bnState == BNState.DROP_UNTIL_NEXT_ROLL) {
       setState(BNState.JOURNAL_ONLY);
     }

Modified: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java?rev=1326020&r1=1326019&r2=1326020&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java Fri Apr 13 23:50:21 2012
@@ -260,7 +260,7 @@ public class BackupNode extends NameNode
     }
 
     /////////////////////////////////////////////////////
-    // BackupNodeProtocol implementation for backup node.
+    // JournalProtocol implementation for backup node.
     /////////////////////////////////////////////////////
     @Override
     public void startLogSegment(JournalInfo journalInfo, long epoch,

Modified: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java?rev=1326020&r1=1326019&r2=1326020&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java Fri Apr 13 23:50:21 2012
@@ -18,18 +18,20 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.common.Util.now;
-import java.net.URI;
+
 import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.lang.reflect.Constructor;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -37,14 +39,34 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CloseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogSegmentOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.conf.Configuration;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -269,7 +291,7 @@ public class FSEditLog  {
       IOUtils.closeStream(s);
     }
     
-    startLogSegment(segmentTxId, true);
+    startLogSegmentAndWriteHeaderTxn(segmentTxId);
     assert state == State.IN_SEGMENT : "Bad state: " + state;
   }
   
@@ -864,18 +886,48 @@ public class FSEditLog  {
     endCurrentLogSegment(true);
     
     long nextTxId = getLastWrittenTxId() + 1;
-    startLogSegment(nextTxId, true);
+    startLogSegmentAndWriteHeaderTxn(nextTxId);
     
     assert curSegmentTxId == nextTxId;
     return nextTxId;
   }
+
+  /**
+   * Remote namenode just has started a log segment, start log segment locally.
+   */
+  public synchronized void startLogSegment(long txid, 
+      boolean abortCurrentLogSegment) throws IOException {
+    LOG.info("Namenode started a new log segment at txid " + txid);
+    if (isSegmentOpen()) {
+      if (getLastWrittenTxId() == txid - 1) {
+        //In sync with the NN, so end and finalize the current segment`
+        endCurrentLogSegment(false);
+      } else {
+        //Missed some transactions: probably lost contact with NN temporarily.
+        final String mess = "Cannot start a new log segment at txid " + txid
+            + " since only up to txid " + getLastWrittenTxId()
+            + " have been written in the log segment starting at "
+            + getCurSegmentTxId() + ".";
+        if (abortCurrentLogSegment) {
+          //Mark the current segment as aborted.
+          LOG.warn(mess);
+          abortCurrentLogSegment();
+        } else {
+          throw new IOException(mess);
+        }
+      }
+    }
+    setNextTxId(txid);
+    startLogSegment(txid);
+  }
   
   /**
    * Start writing to the log segment with the given txid.
    * Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state. 
    */
-  synchronized void startLogSegment(final long segmentTxId,
-      boolean writeHeaderTxn) throws IOException {
+  private void startLogSegment(final long segmentTxId) throws IOException {
+    assert Thread.holdsLock(this);
+
     LOG.info("Starting log segment at " + segmentTxId);
     Preconditions.checkArgument(segmentTxId > 0,
         "Bad txid: %s", segmentTxId);
@@ -903,12 +955,15 @@ public class FSEditLog  {
     
     curSegmentTxId = segmentTxId;
     state = State.IN_SEGMENT;
+  }
 
-    if (writeHeaderTxn) {
-      logEdit(LogSegmentOp.getInstance(cache.get(),
-          FSEditLogOpCodes.OP_START_LOG_SEGMENT));
-      logSync();
-    }
+  synchronized void startLogSegmentAndWriteHeaderTxn(final long segmentTxId
+      ) throws IOException {
+    startLogSegment(segmentTxId);
+
+    logEdit(LogSegmentOp.getInstance(cache.get(),
+        FSEditLogOpCodes.OP_START_LOG_SEGMENT));
+    logSync();
   }
 
   /**
@@ -1057,6 +1112,17 @@ public class FSEditLog  {
     return null;
   }
 
+  /** Write the batch of edits to edit log. */
+  public synchronized void journal(long firstTxId, int numTxns, byte[] data) {
+    final long expectedTxId = getLastWrittenTxId() + 1;
+    Preconditions.checkState(firstTxId == expectedTxId,
+        "received txid batch starting at %s but expected txid %s",
+        firstTxId, expectedTxId);
+    setNextTxId(firstTxId + numTxns - 1);
+    logEdit(data.length, data);
+    logSync();
+  }
+
   /**
    * Write an operation to the edit log. Do not sync to persistent
    * store yet.

Modified: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java?rev=1326020&r1=1326019&r2=1326020&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java Fri Apr 13 23:50:21 2012
@@ -823,7 +823,7 @@ public class FSImage implements Closeabl
       storage.writeAll();
     } finally {
       if (editLogWasOpen) {
-        editLog.startLogSegment(imageTxId + 1, true);
+        editLog.startLogSegmentAndWriteHeaderTxn(imageTxId + 1);
         // Take this opportunity to note the current transaction.
         // Even if the namespace save was cancelled, this marker
         // is only used to determine what transaction ID is required

Propchange: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/native/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native:r1326009-1326019

Propchange: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode:r1326009-1326019

Propchange: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:r1326009-1326019

Propchange: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:r1326009-1326019

Propchange: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs:r1326009-1326019

Modified: hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java?rev=1326020&r1=1326019&r2=1326020&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java Fri Apr 13 23:50:21 2012
@@ -51,7 +51,7 @@ public class TestJournalService {
   static final Log LOG = LogFactory.getLog(TestJournalService.class);
   
   /**
-   * Test calls backs {@link JournalListener#rollLogs(JournalService, long)} and
+   * Test calls backs {@link JournalListener#startLogSegment(JournalService, long)} and
    * {@link JournalListener#journal(JournalService, long, int, byte[])} are
    * called.
    */
@@ -107,7 +107,7 @@ public class TestJournalService {
    */
   private void verifyRollLogsCallback(JournalService s, JournalListener l)
       throws IOException {
-    Mockito.verify(l, Mockito.times(1)).rollLogs(Mockito.eq(s), Mockito.anyLong());
+    Mockito.verify(l, Mockito.times(1)).startLogSegment(Mockito.eq(s), Mockito.anyLong());
   }
 
   /**