Hadoop HDFS
  1. Hadoop HDFS
  2. HDFS-1580

Add interface for generic Write Ahead Logging mechanisms

    Details

    • Type: Improvement Improvement
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 2.0.0-alpha
    • Component/s: namenode
    • Labels:
      None
    • Hadoop Flags:
      Reviewed
    • Target Version/s:
    1. EditlogInterface.1.pdf
      44 kB
      Jitendra Nath Pandey
    2. EditlogInterface.2.pdf
      51 kB
      Jitendra Nath Pandey
    3. EditlogInterface.3.pdf
      65 kB
      Jitendra Nath Pandey
    4. generic_wal_iface.pdf
      158 kB
      Ivan Kelly
    5. generic_wal_iface.pdf
      172 kB
      Ivan Kelly
    6. generic_wal_iface.pdf
      166 kB
      Ivan Kelly
    7. generic_wal_iface.txt
      5 kB
      Ivan Kelly
    8. HDFS-1580.diff
      28 kB
      Ivan Kelly
    9. HDFS-1580.diff
      28 kB
      Ivan Kelly
    10. HDFS-1580.diff
      28 kB
      Ivan Kelly
    11. HDFS-1580.diff
      26 kB
      Ivan Kelly
    12. HDFS-1580.diff
      26 kB
      Ivan Kelly
    13. HDFS-1580.diff
      26 kB
      Ivan Kelly
    14. HDFS-1580.diff
      25 kB
      Ivan Kelly
    15. HDFS-1580.diff
      25 kB
      Ivan Kelly
    16. HDFS-1580.diff
      28 kB
      Ivan Kelly
    17. HDFS-1580.diff
      33 kB
      Ivan Kelly
    18. HDFS-1580.diff
      32 kB
      Ivan Kelly
    19. HDFS-1580.diff
      86 kB
      Ivan Kelly
    20. HDFS-1580+1521.diff
      138 kB
      Ivan Kelly

      Issue Links

        Activity

        Hide
        Ivan Kelly added a comment -

        First draft of generic WAL mechanism, which builds on the NNStorage changes.

        Show
        Ivan Kelly added a comment - First draft of generic WAL mechanism, which builds on the NNStorage changes.
        Hide
        Jitendra Nath Pandey added a comment -

        The interface also needs to have a counterpart of roll-edits method. Currently, for checkpointing, first thing being done is to roll the edit logs i.e. an edits.new is created. As I understand, in hdfs-1073 instead of edits.new the edit files will be numbered. At least FileWriteAheadLog will need to roll to keep edit files from getting too big, even if it is not required for checkpointing.

        The interface should also provide methods to get all previously rotated edit log files (or ledgers) and also current "in-progress" edit log file or ledger.

        As a suggestion, the interface could have a concept of log handles, where each handle uniquely corresponds to single edit log file or ledger. Thus, we could have a method getAllLogs and it will return a list of log-handles.
        I think ordered handles will fit with hdfs-1073 model (need to confirm). LogHandle can also have some meta data for example first transaction id, whether its current or old etc. Hdfs-1073 is proposing to store first transaction-id in the edit-file name itself, which could be used to populate the log-handle in case of FileWriteAheadLog. The input and output streams should be in the LogHandle, so that any log-file can be read. Log-Handle for older files should not let one create an output stream.

        A method to purge the editlogs might also be needed, i.e. given a handle remove the corresponding log-file (or ledger).

        Show
        Jitendra Nath Pandey added a comment - The interface also needs to have a counterpart of roll-edits method. Currently, for checkpointing, first thing being done is to roll the edit logs i.e. an edits.new is created. As I understand, in hdfs-1073 instead of edits.new the edit files will be numbered. At least FileWriteAheadLog will need to roll to keep edit files from getting too big, even if it is not required for checkpointing. The interface should also provide methods to get all previously rotated edit log files (or ledgers) and also current "in-progress" edit log file or ledger. As a suggestion, the interface could have a concept of log handles, where each handle uniquely corresponds to single edit log file or ledger. Thus, we could have a method getAllLogs and it will return a list of log-handles. I think ordered handles will fit with hdfs-1073 model (need to confirm). LogHandle can also have some meta data for example first transaction id, whether its current or old etc. Hdfs-1073 is proposing to store first transaction-id in the edit-file name itself, which could be used to populate the log-handle in case of FileWriteAheadLog. The input and output streams should be in the LogHandle, so that any log-file can be read. Log-Handle for older files should not let one create an output stream. A method to purge the editlogs might also be needed, i.e. given a handle remove the corresponding log-file (or ledger).
        Hide
        Todd Lipcon added a comment -

        Above sounds reasonable with respect to 1073. Is HDFS-1557 almost ready to go? (should I take a look at it?)

        Show
        Todd Lipcon added a comment - Above sounds reasonable with respect to 1073. Is HDFS-1557 almost ready to go? (should I take a look at it?)
        Hide
        Jitendra Nath Pandey added a comment -

        > Is HDFS-1557 almost ready to go? (should I take a look at it?)
        Yes, almost! I am done with my review and Suresh is also taking a look at the patch, once his review is done I will proceed to commit it. You are welcome to take a look.

        Show
        Jitendra Nath Pandey added a comment - > Is HDFS-1557 almost ready to go? (should I take a look at it?) Yes, almost! I am done with my review and Suresh is also taking a look at the patch, once his review is done I will proceed to commit it. You are welcome to take a look.
        Hide
        Ivan Kelly added a comment -

        Addressed some of the comments. Have to discuss the rolling of logs as I'm unsure how 1073 changes it.

        Show
        Ivan Kelly added a comment - Addressed some of the comments. Have to discuss the rolling of logs as I'm unsure how 1073 changes it.
        Hide
        Benjamin Reed added a comment -

        looks good ivan. i like section 1.

        i see a couple of small problems in section 2:

        1) you have a constructor in the interface. you should either make that a comment (meaning that is how you will construct those objects using reflection), or you should call it init(...)

        2) why is transfer() in WriteAheadLogStreamFactory? shouldn't the transfer happen outside of the logging? other logic should be reading from the local WAL and transferring to the secondary.

        3) can you comment a bit more on upgrade(), rollback(), and finalize()?

        Show
        Benjamin Reed added a comment - looks good ivan. i like section 1. i see a couple of small problems in section 2: 1) you have a constructor in the interface. you should either make that a comment (meaning that is how you will construct those objects using reflection), or you should call it init(...) 2) why is transfer() in WriteAheadLogStreamFactory? shouldn't the transfer happen outside of the logging? other logic should be reading from the local WAL and transferring to the secondary. 3) can you comment a bit more on upgrade(), rollback(), and finalize()?
        Hide
        Ivan Kelly added a comment -

        Another update to the interfaces. I've changed it quite a bit. There's still a few open issues, but different ones to yesterday.

        Show
        Ivan Kelly added a comment - Another update to the interfaces. I've changed it quite a bit. There's still a few open issues, but different ones to yesterday.
        Hide
        Jitendra Nath Pandey added a comment -

        > > ShouldWALStreamCustodians control rolling themselves?
        As I understand, book-keeper doesn't allow reading logs from open ledger, is that correct? If that is the case it would be better to first roll and then do the checkpoint to capture most latest edits.

        >WALStreamCustodianNotifier
        It seems to me this will be implemented only by fsedit log. If that is the case we could handle errors just by exceptions i.e. if an operation on WALStreamCustodian interface fails an exception is thrown and fsedit log can decide to remove the custodian depending on the kind of exception.

        > namenode crash
        If the namenode comes back before the znode disappears from zookeeper, the ledger will be open. In that case, will the FSEditLog.load be able to load transactions from the open ledger as well?

        > interface JournalStream
        The document defines this interface but doesn't describe its purpose or use-case.

        > List<URI> getLogs(long sinceTransactionId);
        The list returned must be ordered w.r.t the transactions contained. It might be a good idea to encode the ordering attribute in the url itself, so that the caller of this method can also verify that order is correct. The uri naming convention could mimic this aspect from the convention proposed in 1073.

        > void startRoll()
        > void endRoll()
        I can only imagine a single roll method, that cuts a log, and starts a new one. I believe the naming convention or the ordering attribute for the logs should be controlled by the application and not the storage, therefore the roll method should take a parameter which becomes part of the log metadata and is used to order the logs. Again this also depends on how 1073 does it for file logs.

        Show
        Jitendra Nath Pandey added a comment - > > ShouldWALStreamCustodians control rolling themselves? As I understand, book-keeper doesn't allow reading logs from open ledger, is that correct? If that is the case it would be better to first roll and then do the checkpoint to capture most latest edits. >WALStreamCustodianNotifier It seems to me this will be implemented only by fsedit log. If that is the case we could handle errors just by exceptions i.e. if an operation on WALStreamCustodian interface fails an exception is thrown and fsedit log can decide to remove the custodian depending on the kind of exception. > namenode crash If the namenode comes back before the znode disappears from zookeeper, the ledger will be open. In that case, will the FSEditLog.load be able to load transactions from the open ledger as well? > interface JournalStream The document defines this interface but doesn't describe its purpose or use-case. > List<URI> getLogs(long sinceTransactionId); The list returned must be ordered w.r.t the transactions contained. It might be a good idea to encode the ordering attribute in the url itself, so that the caller of this method can also verify that order is correct. The uri naming convention could mimic this aspect from the convention proposed in 1073. > void startRoll() > void endRoll() I can only imagine a single roll method, that cuts a log, and starts a new one. I believe the naming convention or the ordering attribute for the logs should be controlled by the application and not the storage, therefore the roll method should take a parameter which becomes part of the log metadata and is used to order the logs. Again this also depends on how 1073 does it for file logs.
        Hide
        Ivan Kelly added a comment -

        > > ShouldWALStreamCustodians control rolling themselves?
        As I understand, book-keeper doesn't allow reading logs from open ledger, is that correct? If that is the case it would be better to first roll and then do the checkpoint to capture most latest edits.

        That's true. Therefore there must be a method for having an external entity call roll. However, for the usecase of rolling logs periodically (to keep under a certain size), is there any requirement that en external entity knows anything about it?

        >WALStreamCustodianNotifier
        It seems to me this will be implemented only by fsedit log. If that is the case we could handle errors just by exceptions i.e. if an operation on WALStreamCustodian interface fails an exception is thrown and fsedit log can decide to remove the custodian depending on the kind of exception.

        Also true. In addition to point one, this means we could get rid of WALStreamCustodianNotifier completely.

        > namenode crash
        If the namenode comes back before the znode disappears from zookeeper, the ledger will be open. In that case, will the FSEditLog.load be able to load transactions from the open ledger as well?

        When coming back up, the Bookkeeper WAL implementation will see that there is a ledger open, but no namenode alive, so it will manually close the ledger. At that point FSEditLog will be able to read all the updates.

        > interface JournalStream
        The document defines this interface but doesn't describe its purpose or use-case.

        This interface already exists. I should take these methods out of this design though, as I think 1073 will be adding something like them anyhow.

        > List<URI> getLogs(long sinceTransactionId);
        The list returned must be ordered w.r.t the transactions contained. It might be a good idea to encode the ordering attribute in the url itself, so that the caller of this method can also verify that order is correct. The uri naming convention could mimic this aspect from the convention proposed in 1073.

        Agreed.

        > void startRoll()
        > void endRoll()
        I can only imagine a single roll method, that cuts a log, and starts a new one. I believe the naming convention or the ordering attribute for the logs should be controlled by the application and not the storage, therefore the roll method should take a parameter which becomes part of the log metadata and is used to order the logs. Again this also depends on how 1073 does it for file logs.

        Making the ordering controlled by the application and not the storage makes it hard to encapsulate periodic rolling inside the storage. Rolling requires the current transaction id (i assume this would be the usual parameter) to open a new log.

        However, what we could do in this case, is remove the "roll" call completely. When you want to roll, you just call close on the WALStreamCustodian. Then the next call to getOutputStream() would open a new stream. getOutputStream() would require the current transaction id to know where how to call the new stream, but this shouldn't be a problem.

        Show
        Ivan Kelly added a comment - > > ShouldWALStreamCustodians control rolling themselves? As I understand, book-keeper doesn't allow reading logs from open ledger, is that correct? If that is the case it would be better to first roll and then do the checkpoint to capture most latest edits. That's true. Therefore there must be a method for having an external entity call roll. However, for the usecase of rolling logs periodically (to keep under a certain size), is there any requirement that en external entity knows anything about it? >WALStreamCustodianNotifier It seems to me this will be implemented only by fsedit log. If that is the case we could handle errors just by exceptions i.e. if an operation on WALStreamCustodian interface fails an exception is thrown and fsedit log can decide to remove the custodian depending on the kind of exception. Also true. In addition to point one, this means we could get rid of WALStreamCustodianNotifier completely. > namenode crash If the namenode comes back before the znode disappears from zookeeper, the ledger will be open. In that case, will the FSEditLog.load be able to load transactions from the open ledger as well? When coming back up, the Bookkeeper WAL implementation will see that there is a ledger open, but no namenode alive, so it will manually close the ledger. At that point FSEditLog will be able to read all the updates. > interface JournalStream The document defines this interface but doesn't describe its purpose or use-case. This interface already exists. I should take these methods out of this design though, as I think 1073 will be adding something like them anyhow. > List<URI> getLogs(long sinceTransactionId); The list returned must be ordered w.r.t the transactions contained. It might be a good idea to encode the ordering attribute in the url itself, so that the caller of this method can also verify that order is correct. The uri naming convention could mimic this aspect from the convention proposed in 1073. Agreed. > void startRoll() > void endRoll() I can only imagine a single roll method, that cuts a log, and starts a new one. I believe the naming convention or the ordering attribute for the logs should be controlled by the application and not the storage, therefore the roll method should take a parameter which becomes part of the log metadata and is used to order the logs. Again this also depends on how 1073 does it for file logs. Making the ordering controlled by the application and not the storage makes it hard to encapsulate periodic rolling inside the storage. Rolling requires the current transaction id (i assume this would be the usual parameter) to open a new log. However, what we could do in this case, is remove the "roll" call completely. When you want to roll, you just call close on the WALStreamCustodian. Then the next call to getOutputStream() would open a new stream. getOutputStream() would require the current transaction id to know where how to call the new stream, but this shouldn't be a problem.
        Hide
        Ivan Kelly added a comment -

        Initial work on 1580. This work is on top of 1521. One of the patches includes that, so it will apply on top of trunk. The other requires that 1521 has already been applied.

        TestEditLog is passing. I haven't tried any other tests yet. Most should pass, but anything related to backupnode and secondary will fail for the moment. This is because of a change I've made to construction to FSImage. I've cleaned this up and made setStorageDirectories private in NNStorage. This means that if the storage directories are not read from the configuration file, they must be specified on creation of FSImage. This makes it easier for FSEditLog to know when it can operate on NNStorage and when it cannot. Before it had to wait until someone called setStorageDirectories before doing anything. Very side-effecty. Not FSEditLog knows that if storage exists the storagedirectories have been set. This also allows nicer open close semantics on FSEditLog.

        I plan to move this functionallity into a sub jira as it's nicely separate from the journalling mechanism.

        Show
        Ivan Kelly added a comment - Initial work on 1580. This work is on top of 1521. One of the patches includes that, so it will apply on top of trunk. The other requires that 1521 has already been applied. TestEditLog is passing. I haven't tried any other tests yet. Most should pass, but anything related to backupnode and secondary will fail for the moment. This is because of a change I've made to construction to FSImage. I've cleaned this up and made setStorageDirectories private in NNStorage. This means that if the storage directories are not read from the configuration file, they must be specified on creation of FSImage. This makes it easier for FSEditLog to know when it can operate on NNStorage and when it cannot. Before it had to wait until someone called setStorageDirectories before doing anything. Very side-effecty. Not FSEditLog knows that if storage exists the storagedirectories have been set. This also allows nicer open close semantics on FSEditLog. I plan to move this functionallity into a sub jira as it's nicely separate from the journalling mechanism.
        Hide
        Todd Lipcon added a comment -

        Hi Ivan. I'm trying to figure out how this work will fit in with the log-transfer step between checkpointers and the primary NN. Do you anticipate that these JournalManager implementations will be instantiated on the checkpointer side as well? Or is the transfer method orthogonal to the storage method, and this JIRA is only about the NN side of things?

        It seems to me they have to be orthogonal concerns, at least initially, if you plan to be able to have two separate "types" of edit logs on the NN. So, even if the NN is using both BK and files, a 2NN would still use HTTP to fetch logs at checkpoint time.

        In working on HDFS-1073, I'm thinking the interface for log transfer will look something like:

        • add a new RPC method to the NN corresponding to your getLogs(long sinceTxId). This would return something like List<RemoteLog>, where RemoteLog has some method which takes care of fetching.
        • for now, when the 2NN wants to checkpoint, it initiates rollEdits() which returns CheckpointSignature. This would be modified to include the lastCheckpointTxId field
        • the 2NN then issues getLogs(sig.lastCheckpointTxId), iterates over each of the RemoteLogs, and fetches them

        Does this gel with what you're planning?

        Show
        Todd Lipcon added a comment - Hi Ivan. I'm trying to figure out how this work will fit in with the log-transfer step between checkpointers and the primary NN. Do you anticipate that these JournalManager implementations will be instantiated on the checkpointer side as well? Or is the transfer method orthogonal to the storage method, and this JIRA is only about the NN side of things? It seems to me they have to be orthogonal concerns, at least initially, if you plan to be able to have two separate "types" of edit logs on the NN. So, even if the NN is using both BK and files, a 2NN would still use HTTP to fetch logs at checkpoint time. In working on HDFS-1073 , I'm thinking the interface for log transfer will look something like: add a new RPC method to the NN corresponding to your getLogs(long sinceTxId). This would return something like List<RemoteLog>, where RemoteLog has some method which takes care of fetching. for now, when the 2NN wants to checkpoint, it initiates rollEdits() which returns CheckpointSignature. This would be modified to include the lastCheckpointTxId field the 2NN then issues getLogs(sig.lastCheckpointTxId), iterates over each of the RemoteLogs, and fetches them Does this gel with what you're planning?
        Hide
        Ivan Kelly added a comment -

        To read in a log you pass in a URI to the getInputStream method. For BK, this will allow us to have no explicit transfer stage. Files are a different matter. I haven't thought much about it yet, but file URI do have a space for host. file://<host>/<path>. The FileJournalManager could return a URI with the host part set for getLogs. This could be passed to the getInputStream call, which would download it from the remote host if nescessary.

        So, the steps would be as you outlined except for the "iterates over each of the RemoteLogs, and fetches them". Instead you would just iterate over them and load them using the JournalManager.getInputStream method. The implementation of the method would take care of whether the file is remote or local.

        Show
        Ivan Kelly added a comment - To read in a log you pass in a URI to the getInputStream method. For BK, this will allow us to have no explicit transfer stage. Files are a different matter. I haven't thought much about it yet, but file URI do have a space for host. file:// <host>/<path>. The FileJournalManager could return a URI with the host part set for getLogs. This could be passed to the getInputStream call, which would download it from the remote host if nescessary. So, the steps would be as you outlined except for the "iterates over each of the RemoteLogs, and fetches them". Instead you would just iterate over them and load them using the JournalManager.getInputStream method. The implementation of the method would take care of whether the file is remote or local.
        Hide
        Todd Lipcon added a comment -

        Hi Ivan,

        Trying to integrate this with my HDFS-1073 work, running into some things that feel wrong about the JournalManager API...
        It seems strange that every edit makes a call to getOutputStream(txid) rather than simply calling it once at each roll point.

        I think it makes the most sense to introduce the idea of a "log segment" - in the file case it corresponds to a single edit file. In the BK case it might be a ledger. In other cases it might be a no-op.

        The state machine for logging then becomes something like:

        • open
        • beginLogSegment(txid 1): returns an EditLogOutputStream
        • write(...) 5 times
        • roll:
          • endLogSegment(1, 6)
          • startLogSegment(7)
        • write more data, etc

        Does this seem to make sense to you?

        Show
        Todd Lipcon added a comment - Hi Ivan, Trying to integrate this with my HDFS-1073 work, running into some things that feel wrong about the JournalManager API... It seems strange that every edit makes a call to getOutputStream(txid) rather than simply calling it once at each roll point. I think it makes the most sense to introduce the idea of a "log segment" - in the file case it corresponds to a single edit file. In the BK case it might be a ledger. In other cases it might be a no-op. The state machine for logging then becomes something like: open beginLogSegment(txid 1): returns an EditLogOutputStream write(...) 5 times roll: endLogSegment(1, 6) startLogSegment(7) write more data, etc Does this seem to make sense to you?
        Hide
        Ivan Kelly added a comment -

        This depends on whether we want the roll operation exposed to the user of the JournalManager. In the past this was required as roll had to happen to allow a checkpoint to take place. However, once HDFS-1580 goes in, its not strictly necessary because rolling is stateless. The JournalManager could roll in the background without the user of the class having any knowledge of the roll. To the user, the JournalManager is simply something to open, write to and close. Do you think this simplification is worthwhile?

        I agree that getOutputStream(txid) everywhere is a bit messy. It also doesnt work. If a roll happens while the OutputStream has been given out the editlog may end up writing to two different streams which could mess up transaction ids. This isn't in any way unsolvable, but I need to think about it more.

        Show
        Ivan Kelly added a comment - This depends on whether we want the roll operation exposed to the user of the JournalManager. In the past this was required as roll had to happen to allow a checkpoint to take place. However, once HDFS-1580 goes in, its not strictly necessary because rolling is stateless. The JournalManager could roll in the background without the user of the class having any knowledge of the roll. To the user, the JournalManager is simply something to open, write to and close. Do you think this simplification is worthwhile? I agree that getOutputStream(txid) everywhere is a bit messy. It also doesnt work. If a roll happens while the OutputStream has been given out the editlog may end up writing to two different streams which could mess up transaction ids. This isn't in any way unsolvable, but I need to think about it more.
        Hide
        Todd Lipcon added a comment -

        I'm not sure I agree with your point that rolling should be a concern of the JournalManager and not of the higher-level frameworks. In HDFS-1073, Sanjay and I had come to the conclusion that we should maintain the invariant that different edits logs should roll at the same time. This makes it possible to take two edits directories (even across the BN and NN) and the files inside should be md5sum-identical to each other – very important for ops and peace of mind.

        Perhaps some WAL implementions may want to ignore the roll boundaries, but we at least need coordination between different File logs.

        Show
        Todd Lipcon added a comment - I'm not sure I agree with your point that rolling should be a concern of the JournalManager and not of the higher-level frameworks. In HDFS-1073 , Sanjay and I had come to the conclusion that we should maintain the invariant that different edits logs should roll at the same time. This makes it possible to take two edits directories (even across the BN and NN) and the files inside should be md5sum-identical to each other – very important for ops and peace of mind. Perhaps some WAL implementions may want to ignore the roll boundaries, but we at least need coordination between different File logs.
        Hide
        Jitendra Nath Pandey added a comment -

        A few comments.

        1. > JournalManager#transfer()
        Is this a push method to be executed on primary or a pull from secondary? The mechanism as we have today, which is more like a pull, can be provided by the interface using getLogs and getInputStream methods, which, in my opinion, is sufficient.

        2. We could use single BookKeeper installation (or any shared storage solution) for storing edit logs from multiple namenodes (e.g. federation). In that case a secondary/backup should be able to instantiate a JournalManager and request logs for a specific namenode. Similar could be argued about the files on namenodes, where backup or secondary wants to specify the namenode where it wants to download the files from.
        As I understand the steps to get edit logs at the backup node or secondary would be

        • Instantiate a JournalManager
        • specify the source of edit logs. This could be specified in the JournalManager constructor. In case of BK the source would be the set of ledgers for a specific namenode.
        • getLogs from sinceTransactionId. It returns a list of URIs.
        • For each uri get the input stream and read the logs.

        3. For each edit log we need to store some metadata, particularly the LAYOUT_VERSION. I think it would make sense to provide for that in JournalManager interface.

        4. For edit logs do we really need upgrade, rollback or finalizeUpgrade? For edit log files, it seems to me that all that matters is whether the namenode has right version to understand them or load them, which can be determined from LAYOUT_VERSION.

        5. I will recommend putting JournalManager and its derived classes in separate packages.

        6. Please add javadoc comments on JournalManager class and its methods.

        Show
        Jitendra Nath Pandey added a comment - A few comments. 1. > JournalManager#transfer() Is this a push method to be executed on primary or a pull from secondary? The mechanism as we have today, which is more like a pull, can be provided by the interface using getLogs and getInputStream methods, which, in my opinion, is sufficient. 2. We could use single BookKeeper installation (or any shared storage solution) for storing edit logs from multiple namenodes (e.g. federation). In that case a secondary/backup should be able to instantiate a JournalManager and request logs for a specific namenode. Similar could be argued about the files on namenodes, where backup or secondary wants to specify the namenode where it wants to download the files from. As I understand the steps to get edit logs at the backup node or secondary would be Instantiate a JournalManager specify the source of edit logs. This could be specified in the JournalManager constructor. In case of BK the source would be the set of ledgers for a specific namenode. getLogs from sinceTransactionId. It returns a list of URIs. For each uri get the input stream and read the logs. 3. For each edit log we need to store some metadata, particularly the LAYOUT_VERSION. I think it would make sense to provide for that in JournalManager interface. 4. For edit logs do we really need upgrade, rollback or finalizeUpgrade? For edit log files, it seems to me that all that matters is whether the namenode has right version to understand them or load them, which can be determined from LAYOUT_VERSION. 5. I will recommend putting JournalManager and its derived classes in separate packages. 6. Please add javadoc comments on JournalManager class and its methods.
        Hide
        Ivan Kelly added a comment -

        @Todd
        Ah ok, since rolling has to be exposed, we should expose it fully. Your segment suggestion sounds good. It will require a separate array in FSEditLog though, one for JournalManagers, one for EditLogOutputStreams. I'll have to think about this. Perhaps we could get rid of the journal manager abstraction completely and extend EditLogOutputStream to handle rolling, as rolling is the only operation that changes for output with transactions.

        Then we could have a separate class that handles stuff like format and getting a list of the logs. I'll think about this more and put up a new design.

        @Jitendra
        1. I think I mentioned the second alternative earlier in the thread, but yes, #transfer will go away.
        2. A single bookkeeper setup can be shared by many writers. The more writers you have the slower the reads, as ledgers are interleaved when written to disk.
        3. There's two layouts to think about, the storage layout and the data layout. The first will be entirely internal to the JournalManager. The second will need to be saved though.
        4. Very good point.
        5. Ok
        6. Will do

        Show
        Ivan Kelly added a comment - @Todd Ah ok, since rolling has to be exposed, we should expose it fully. Your segment suggestion sounds good. It will require a separate array in FSEditLog though, one for JournalManagers, one for EditLogOutputStreams. I'll have to think about this. Perhaps we could get rid of the journal manager abstraction completely and extend EditLogOutputStream to handle rolling, as rolling is the only operation that changes for output with transactions. Then we could have a separate class that handles stuff like format and getting a list of the logs. I'll think about this more and put up a new design. @Jitendra 1. I think I mentioned the second alternative earlier in the thread, but yes, #transfer will go away. 2. A single bookkeeper setup can be shared by many writers. The more writers you have the slower the reads, as ledgers are interleaved when written to disk. 3. There's two layouts to think about, the storage layout and the data layout. The first will be entirely internal to the JournalManager. The second will need to be saved though. 4. Very good point. 5. Ok 6. Will do
        Hide
        Ivan Kelly added a comment -

        Changed interfaces. Design is now more based around EditLogOutputStream. Also added a section about transfer of logs across nodes.

        Show
        Ivan Kelly added a comment - Changed interfaces. Design is now more based around EditLogOutputStream. Also added a section about transfer of logs across nodes.
        Hide
        Jitendra Nath Pandey added a comment -

        Ivan,
        I am thinking of something like following code. I am using name TxnLogOutputStream instead of EditLogOutputStream because the later confuses with existing EditLogOutputStream in Hdfs. This undoes some of my previous suggestions e.g. regarding putting transaction id in the URI. It seems better that the Journal is viewed as a single input or output stream and URI just identifies the namespace.
        I think we should use TxnRecord object which encapsulate a transaction as a byte-array instead of using Writables, its better to keep interface unaware of serialization protocol.

        
        abstract class JournalWriter {
             //The URI identifies the journal location & namespace
             JournalWriter(URI u);
        
             //Initializes the layoutVersion
             void format(int layoutVersion);
        
             //After this call, any subsequent transactions should
             //map to newLayoutVersion, to be used in upgrade or rollback.
             void setLayoutVersion(int newLayoutVersion);
        
             TxnLogOutputStream getOutputStream();
        
             //Deletes all transactions older than uptoTxnId (inclusive)
             void delete(long uptoTxnId);
        }
        class TxnRecord {
           long txnId;
           byte[] txn;
        }
        interface TxnLogOutputStream {
             write(long txnId, TxnRecord rec);
             void roll();
             close();
        }
        abstract class JournalReader {
             JournalReader(URI u);
        
             TxnLogInputStream getInputStream(long sinceTxnId);
        }
        interface TxnLogInputStream {
             TxnRecord next();
        
             //Returns layout version of the next record.
             int getLayoutVersion();
             void close();
        }
        
        
        Show
        Jitendra Nath Pandey added a comment - Ivan, I am thinking of something like following code. I am using name TxnLogOutputStream instead of EditLogOutputStream because the later confuses with existing EditLogOutputStream in Hdfs. This undoes some of my previous suggestions e.g. regarding putting transaction id in the URI. It seems better that the Journal is viewed as a single input or output stream and URI just identifies the namespace. I think we should use TxnRecord object which encapsulate a transaction as a byte-array instead of using Writables, its better to keep interface unaware of serialization protocol. abstract class JournalWriter { //The URI identifies the journal location & namespace JournalWriter(URI u); //Initializes the layoutVersion void format( int layoutVersion); //After this call, any subsequent transactions should //map to newLayoutVersion, to be used in upgrade or rollback. void setLayoutVersion( int newLayoutVersion); TxnLogOutputStream getOutputStream(); //Deletes all transactions older than uptoTxnId (inclusive) void delete( long uptoTxnId); } class TxnRecord { long txnId; byte [] txn; } interface TxnLogOutputStream { write( long txnId, TxnRecord rec); void roll(); close(); } abstract class JournalReader { JournalReader(URI u); TxnLogInputStream getInputStream( long sinceTxnId); } interface TxnLogInputStream { TxnRecord next(); //Returns layout version of the next record. int getLayoutVersion(); void close(); }
        Hide
        Ivan Kelly added a comment -

        I'd like to keep the pluggable bit (JournalFactory, or JournalWriter&JournalReader) to one interface for ease of configuration. Otherwise when a user specifies a custom journal type in their configuration, they'll have to specify the class for the Writer and for the Reader.

        The EditLogOutputStream i proposed won't coexist with the current EditLogOutputStream, it will replace it. We need a single interface for writing the edit log, so that FSEditLog can loop through all the streams and treat them the same.

        I like the idea of TxnRecord.

        Show
        Ivan Kelly added a comment - I'd like to keep the pluggable bit (JournalFactory, or JournalWriter&JournalReader) to one interface for ease of configuration. Otherwise when a user specifies a custom journal type in their configuration, they'll have to specify the class for the Writer and for the Reader. The EditLogOutputStream i proposed won't coexist with the current EditLogOutputStream, it will replace it. We need a single interface for writing the edit log, so that FSEditLog can loop through all the streams and treat them the same. I like the idea of TxnRecord.
        Hide
        Todd Lipcon added a comment -

        I'm not sure about TxnRecord - if not done carefully it will cause an extra memory copy on all the writes, which isn't free. I like modularity, but we will have to carefully benchmark if we make that change.

        Show
        Todd Lipcon added a comment - I'm not sure about TxnRecord - if not done carefully it will cause an extra memory copy on all the writes, which isn't free. I like modularity, but we will have to carefully benchmark if we make that change.
        Hide
        Todd Lipcon added a comment -

        Hi Ivan and Jitendra. I've posted a patch for the edit log branch to HDFS-1799 that is a subset of this JIRA. It's different in that it doesn't make the interface pluggable, nor does it cover log transfer capability yet. But you may be interested to take a look at that and see if the interface seems to be headed in a direction that's compatible with your goals. I think it should be – I started by merging your patch here and then ripped things out until it was the minimal change to just refactor the file-specific code out of FSEditLog as much as possible.

        Show
        Todd Lipcon added a comment - Hi Ivan and Jitendra. I've posted a patch for the edit log branch to HDFS-1799 that is a subset of this JIRA. It's different in that it doesn't make the interface pluggable, nor does it cover log transfer capability yet. But you may be interested to take a look at that and see if the interface seems to be headed in a direction that's compatible with your goals. I think it should be – I started by merging your patch here and then ripped things out until it was the minimal change to just refactor the file-specific code out of FSEditLog as much as possible.
        Hide
        Ivan Kelly added a comment -

        First drop of implementation of most recent design. Changes are intended to be minimal. Compiles, but doesn't pass tests yet.

        Show
        Ivan Kelly added a comment - First drop of implementation of most recent design. Changes are intended to be minimal. Compiles, but doesn't pass tests yet.
        Hide
        Ivan Kelly added a comment -

        Applies on top of 1073 branch.

        Show
        Ivan Kelly added a comment - Applies on top of 1073 branch.
        Hide
        Jitendra Nath Pandey added a comment -

        I have uploaded a new design document for editlog interface. It is based on Ivan's earlier proposal and a series of discussions I have had with Sanjay, Suresh and Ivan.

        The requirement for snapshots is not addressed in this document. The idea is that this interface can later be extended to support a "SnapshotableJournal"

        Show
        Jitendra Nath Pandey added a comment - I have uploaded a new design document for editlog interface. It is based on Ivan's earlier proposal and a series of discussions I have had with Sanjay, Suresh and Ivan. The requirement for snapshots is not addressed in this document. The idea is that this interface can later be extended to support a "SnapshotableJournal"
        Hide
        Todd Lipcon added a comment -

        Hi Jitendra. Here are some thoughts on your latest document:

        • While I appreciate that this work will probably make snapshots a little easier down the road, it's by far not the most difficult part of supporting snapshots, nor is it really the goal we're trying to address. So I think it's premature to mention snapshots in the design.
        • The concept of "layout version" I think has been overloaded way too much. We currently use a single version number to indicate (a) the file and serialziation format for image dumps, (b) the file and serialization format for edit logs, and (c) the actual layout of files within the current/ directory. I would like to advocate splitting this out into IMAGE_FORMAT_VERSION, EDITS_FORMAT_VERSION, and LAYOUT_VERSION. To be clear, this jira is mostly concerned with what I would call EDITS_FORMAT_VERSION (e.g. the way in which we turn a mkdirs into bytes). Do you agree with this interpretation?
        • The idea of a purgeTransactions call makes sense – after a checkpoint has been uploaded for txid N, we don't need edits prior to N. However, there are some policies that make sense to me like "keep edits for at least a week". Would you assume these retention policies would be the responsibility of the edit log implementation? ie that, even if told to purge transactions older than txid N, it might keep them around for some time, or take care of archiving them to a NAS/HDFS?
        • For the getInputStream call, is there any restriction on valid values of sinceTxId that it be on any kind of boundary? e.g that it must correspond to a "mark" call? See more about this below regarding the idea of "log segments"
        • I don't entirely understand the usage of the setVersion call. When would the version of a log change mid-stream?
        • I'm not entirely clear on "mark" as well. The semantics described in the "Discussion" section are what I would normally call sync, but in other parts of the document it's described as a roll equivalent. If it's not sync, then we're missing sync altogether, and that implies that each write call will have to sync on its own, thus breaking group commit. I think we should maintain the existing buffering/syncing calls write, setReadyToFlush, and flushAndSync.
        • The EditLogInputStream interface is strange - it's called InputStream but doesn't follow a normal InputStream API. It's something sort of like an Iterator, but also doesn't implement that interface. Could we add a wrapper class EditTransaction, and make EditLogInputStream an Interable<EditTransaction>? EditTransaction would then take the getTxnId call.
        • The API getTxn shouldn't return byte[] since that implies an extra buffer copy to get a transaction into its own array. Instead it should be able to point into an existing byte array. Alternatively, the input stream could continue to implement InputStream so we can use the existing editlog loading code.

        As I've proposed over in some other JIRAs, I think we should do away with the roll call, and instead make the concept of log segments a first class citizen. In the file-based storage case, a log segment is an individual file. In the BK case, it may be that a log segment is a ledger (I don't know BK's API well).

        Thus, rolling the logs becomes a sequence like:

            endCurrentLogSegment();
            long nextTxId = getLastWrittenTxId() + 1;
            LOG.info("Rolling edit logs. Next txid after roll will be " + nextTxId);
            startLogSegment(nextTxId);
        

        where endCurrentLogSegment closes off the current segment across all journals, and startLogSegment starts a new output stream across all journals.

        The advantages I see of this approach are:

        • elsewhere we have discussed that we want to keep the property that logs always roll together across all parts of the system, and thus that the storage directories have parallel contents with identical names and identical file contents. It's possible to achieve this with just the roll API, but it becomes more obvious how to do it with the segment concept. As one example, consider what happens when one journal fails (eg due to an NFS mount going down temporarily). While it's down, we don't write txns to this journal. But, after some time we may notice that the mount is available again. Rather than just calling roll here, it makes sense to be explicit that we're starting a new segment, and be explicit about the starting txid of that new segment.
        • We generally want the property that, while saving a namespace or in safe mode, we don't accept edits. Thus, it would be nice to have the edit log actually be closed during this operation. Splitting roll into a endCurrent and startNext allows us to add the namespace dump between the two and make sure that no edits could possibly be written while saving.

        What do you think about these suggestions? You can see a working tree with the "log segment" concept at https://github.com/toddlipcon/hadoop-hdfs/tree/hdfs-1073-march/src/java/org/apache/hadoop/hdfs/server/namenode/

        Show
        Todd Lipcon added a comment - Hi Jitendra. Here are some thoughts on your latest document: While I appreciate that this work will probably make snapshots a little easier down the road, it's by far not the most difficult part of supporting snapshots, nor is it really the goal we're trying to address. So I think it's premature to mention snapshots in the design. The concept of "layout version" I think has been overloaded way too much. We currently use a single version number to indicate (a) the file and serialziation format for image dumps, (b) the file and serialization format for edit logs, and (c) the actual layout of files within the current/ directory. I would like to advocate splitting this out into IMAGE_FORMAT_VERSION, EDITS_FORMAT_VERSION, and LAYOUT_VERSION. To be clear, this jira is mostly concerned with what I would call EDITS_FORMAT_VERSION (e.g. the way in which we turn a mkdirs into bytes). Do you agree with this interpretation? The idea of a purgeTransactions call makes sense – after a checkpoint has been uploaded for txid N, we don't need edits prior to N. However, there are some policies that make sense to me like "keep edits for at least a week". Would you assume these retention policies would be the responsibility of the edit log implementation? ie that, even if told to purge transactions older than txid N, it might keep them around for some time, or take care of archiving them to a NAS/HDFS? For the getInputStream call, is there any restriction on valid values of sinceTxId that it be on any kind of boundary? e.g that it must correspond to a "mark" call? See more about this below regarding the idea of "log segments" I don't entirely understand the usage of the setVersion call. When would the version of a log change mid-stream? I'm not entirely clear on "mark" as well. The semantics described in the "Discussion" section are what I would normally call sync , but in other parts of the document it's described as a roll equivalent. If it's not sync, then we're missing sync altogether, and that implies that each write call will have to sync on its own, thus breaking group commit. I think we should maintain the existing buffering/syncing calls write , setReadyToFlush , and flushAndSync . The EditLogInputStream interface is strange - it's called InputStream but doesn't follow a normal InputStream API. It's something sort of like an Iterator, but also doesn't implement that interface. Could we add a wrapper class EditTransaction , and make EditLogInputStream an Interable<EditTransaction>? EditTransaction would then take the getTxnId call. The API getTxn shouldn't return byte[] since that implies an extra buffer copy to get a transaction into its own array. Instead it should be able to point into an existing byte array. Alternatively, the input stream could continue to implement InputStream so we can use the existing editlog loading code. As I've proposed over in some other JIRAs, I think we should do away with the roll call, and instead make the concept of log segments a first class citizen. In the file-based storage case, a log segment is an individual file. In the BK case, it may be that a log segment is a ledger (I don't know BK's API well). Thus, rolling the logs becomes a sequence like: endCurrentLogSegment(); long nextTxId = getLastWrittenTxId() + 1; LOG.info( "Rolling edit logs. Next txid after roll will be " + nextTxId); startLogSegment(nextTxId); where endCurrentLogSegment closes off the current segment across all journals, and startLogSegment starts a new output stream across all journals. The advantages I see of this approach are: elsewhere we have discussed that we want to keep the property that logs always roll together across all parts of the system, and thus that the storage directories have parallel contents with identical names and identical file contents. It's possible to achieve this with just the roll API, but it becomes more obvious how to do it with the segment concept. As one example, consider what happens when one journal fails (eg due to an NFS mount going down temporarily). While it's down, we don't write txns to this journal. But, after some time we may notice that the mount is available again. Rather than just calling roll here, it makes sense to be explicit that we're starting a new segment, and be explicit about the starting txid of that new segment. We generally want the property that, while saving a namespace or in safe mode, we don't accept edits. Thus, it would be nice to have the edit log actually be closed during this operation. Splitting roll into a endCurrent and startNext allows us to add the namespace dump between the two and make sure that no edits could possibly be written while saving. What do you think about these suggestions? You can see a working tree with the "log segment" concept at https://github.com/toddlipcon/hadoop-hdfs/tree/hdfs-1073-march/src/java/org/apache/hadoop/hdfs/server/namenode/
        Hide
        Jitendra Nath Pandey added a comment -
        • The design doesn't go in any detail regarding snapshots concurring to your view. However, I mentioned about it because it is one of the requirements we will have to address eventually.
        • This jira doesn't change any semantics related to the layout version. The version is a piece of metadata that needs to be stored with edit logs so that namenode can understand and load edit logs. I am open to making it a byte array instead of just an integer so that namenode can store any metadata it wants to store, which is relevant for understanding the edit logs. I agree that version is a little overloaded but that can be addressed in a different jira.
        • I think retention policy for edit logs should be namenode's responsibility, because retention of edit logs will be closely tied with retention of old checkpoint images. If namenode has called purgeTransactions it should never ask for older transaction ids.
        • "mark" means that the last written transaction is available for reading including all previous transactions. sinceTxnId in getInputStream can be any transaction Id before the last call of mark or close of the output stream. Apart from that, sinceTxnId doesn't assume any boundary.
        • The motivation for "mark" method was that BK has this limitation that open ledgers cannot be read, "mark" will give a cue to a BK implementation that the current ledger should be made available for reading. If an implementation doesn't have this limitation it can just ignore mark, that is why I didn't call it roll. That also explains that it is different from sync.
        • I assumed that a write also syncs, because in most operations we sync immediately after writing the log, and in this design we are writing the entire transaction as a unit. Management of buffers and flush, should be the responsibility of the implementation.
        • In EditLogInputStream, I think we can rename next to readNext, it will look less like iterator. One way to avoid extra array copy would be that readNext() reads the version and txnId and synchronizes the underlying inputstream to the begining of transaction record and then getTxn can directly return the underlying inputstream for reading the transaction bytes. Does that make sense?

        LogSegements:
        LogSegments gets rid of roll method but exposes the underlying units of storage to the namenode which I don't think is required.

        >.. elsewhere we have discussed that we want to keep the property that logs always roll together across all parts of the system.
        Do we really want this property? Isn't it better that we don't expose any boundaries between transactions to the namenode?
        > We generally want the property that, while saving a namespace or in safe mode, we don't accept edits.
        This can be achieved by just closing the EditLogOutputStream.

        Show
        Jitendra Nath Pandey added a comment - The design doesn't go in any detail regarding snapshots concurring to your view. However, I mentioned about it because it is one of the requirements we will have to address eventually. This jira doesn't change any semantics related to the layout version. The version is a piece of metadata that needs to be stored with edit logs so that namenode can understand and load edit logs. I am open to making it a byte array instead of just an integer so that namenode can store any metadata it wants to store, which is relevant for understanding the edit logs. I agree that version is a little overloaded but that can be addressed in a different jira. I think retention policy for edit logs should be namenode's responsibility, because retention of edit logs will be closely tied with retention of old checkpoint images. If namenode has called purgeTransactions it should never ask for older transaction ids. "mark" means that the last written transaction is available for reading including all previous transactions. sinceTxnId in getInputStream can be any transaction Id before the last call of mark or close of the output stream. Apart from that, sinceTxnId doesn't assume any boundary. The motivation for "mark" method was that BK has this limitation that open ledgers cannot be read, "mark" will give a cue to a BK implementation that the current ledger should be made available for reading. If an implementation doesn't have this limitation it can just ignore mark, that is why I didn't call it roll. That also explains that it is different from sync. I assumed that a write also syncs, because in most operations we sync immediately after writing the log, and in this design we are writing the entire transaction as a unit. Management of buffers and flush, should be the responsibility of the implementation. In EditLogInputStream, I think we can rename next to readNext, it will look less like iterator. One way to avoid extra array copy would be that readNext() reads the version and txnId and synchronizes the underlying inputstream to the begining of transaction record and then getTxn can directly return the underlying inputstream for reading the transaction bytes. Does that make sense? LogSegements: LogSegments gets rid of roll method but exposes the underlying units of storage to the namenode which I don't think is required. >.. elsewhere we have discussed that we want to keep the property that logs always roll together across all parts of the system. Do we really want this property? Isn't it better that we don't expose any boundaries between transactions to the namenode? > We generally want the property that, while saving a namespace or in safe mode, we don't accept edits. This can be achieved by just closing the EditLogOutputStream.
        Hide
        Todd Lipcon added a comment -

        which is relevant for understanding the edit logs. I agree that version is a little overloaded but that can be addressed in a different jira

        Agreed that's a separate JIRA – I just wanted to clarify that the version you're talking about here is the "edits log serialization format version" rather than something about actual layout.

        If namenode has called purgeTransactions it should never ask for older transaction ids

        Fair enough.

        Apart from that, sinceTxnId doesn't assume any boundary

        I think that will really complicate things like edits transfer in the 2NN. In the file-based storage there's no clean way to seek to a particular transaction ID, meaning we'd have to add in this facility into EditLogInputStream, etc. That's a lot of complexity for little benefit that I can see.

        The motivation for "mark" method was that BK has this limitation that open ledgers cannot be read, "mark" will give a cue to a BK implementation that the current ledger should be made available for reading

        This seems like a somewhat serious flaw. If we anticipate using BK for HA, I was under the impression that the "hot backup" would be following along on the edits as they're written into BK. What you're saying here implies that the primary NN would have to be rolling its logs every few seconds if you want the standby to be truly "hot".

        If an implementation doesn't have this limitation it can just ignore mark, that is why I didn't call it roll

        Another way of doing this is to say that, if an implementation does have this limitation, it can choose to "mark" whenever it likes. No?

        I assumed that a write also syncs, because in most operations we sync immediately after writing the log, and in this design we are writing the entire transaction as a unit.

        In fact this is not at all how the current design works. Most operations write the edit to the log while holding the FSN lock (to ensure serialized order between ops) and then drop the FSN lock to sync. This allows group commit and is crucial for reasonable throughput.

        Management of buffers and flush, should be the responsibility of the implementation.

        But flush needs to be coordinated as a separate action from writing in order to achieve lock release and group commit.

        readNext() reads the version and txnId and synchronizes the underlying inputstream to the begining of transaction record and then getTxn can directly return the underlying inputstream for reading the transaction bytes

        Yep, that makes sense.

        LogSegments gets rid of roll method but exposes the underlying units of storage to the namenode which I don't think is required

        It's not absolutely required in the theoretical sense, but in the sense that we'd like to keep the code as simple as possible, I think it helps that goal. For example, edit log transfer right now is based around the concept of discrete files which can be entirely fetched, with an associated md5sum. If we have to support fetching arbitrary ranges of transactions, these safety checks become more difficult to implement. And, we need to split the "file transfer" code into two different code paths, one for files (fsimage) and another for edits (arbitrary transaction ranges)

        Do we really want this property? Isn't it better that we don't expose any boundaries between transactions to the namenode?

        Yes, this property is very useful for operations. Refer to the discussion on HDFS-1073 about this property. The fact that I can run "md5sum /data/

        {1..4}

        /dfs/name/current/*" and verify that the files are all identical gives me great peace of mind.

        Show
        Todd Lipcon added a comment - which is relevant for understanding the edit logs. I agree that version is a little overloaded but that can be addressed in a different jira Agreed that's a separate JIRA – I just wanted to clarify that the version you're talking about here is the "edits log serialization format version" rather than something about actual layout. If namenode has called purgeTransactions it should never ask for older transaction ids Fair enough. Apart from that, sinceTxnId doesn't assume any boundary I think that will really complicate things like edits transfer in the 2NN. In the file-based storage there's no clean way to seek to a particular transaction ID, meaning we'd have to add in this facility into EditLogInputStream, etc. That's a lot of complexity for little benefit that I can see. The motivation for "mark" method was that BK has this limitation that open ledgers cannot be read, "mark" will give a cue to a BK implementation that the current ledger should be made available for reading This seems like a somewhat serious flaw. If we anticipate using BK for HA, I was under the impression that the "hot backup" would be following along on the edits as they're written into BK. What you're saying here implies that the primary NN would have to be rolling its logs every few seconds if you want the standby to be truly "hot". If an implementation doesn't have this limitation it can just ignore mark, that is why I didn't call it roll Another way of doing this is to say that, if an implementation does have this limitation, it can choose to "mark" whenever it likes. No? I assumed that a write also syncs, because in most operations we sync immediately after writing the log, and in this design we are writing the entire transaction as a unit. In fact this is not at all how the current design works. Most operations write the edit to the log while holding the FSN lock (to ensure serialized order between ops) and then drop the FSN lock to sync. This allows group commit and is crucial for reasonable throughput. Management of buffers and flush, should be the responsibility of the implementation. But flush needs to be coordinated as a separate action from writing in order to achieve lock release and group commit. readNext() reads the version and txnId and synchronizes the underlying inputstream to the begining of transaction record and then getTxn can directly return the underlying inputstream for reading the transaction bytes Yep, that makes sense. LogSegments gets rid of roll method but exposes the underlying units of storage to the namenode which I don't think is required It's not absolutely required in the theoretical sense, but in the sense that we'd like to keep the code as simple as possible, I think it helps that goal. For example, edit log transfer right now is based around the concept of discrete files which can be entirely fetched, with an associated md5sum. If we have to support fetching arbitrary ranges of transactions, these safety checks become more difficult to implement. And, we need to split the "file transfer" code into two different code paths, one for files (fsimage) and another for edits (arbitrary transaction ranges) Do we really want this property? Isn't it better that we don't expose any boundaries between transactions to the namenode? Yes, this property is very useful for operations. Refer to the discussion on HDFS-1073 about this property. The fact that I can run "md5sum /data/ {1..4} /dfs/name/current/*" and verify that the files are all identical gives me great peace of mind.
        Hide
        Jitendra Nath Pandey added a comment -

        > In the file-based storage there's no clean way to seek to a particular transaction ID
        A savenamespace will be preceded by a call to mark (like current roll). A file implementation can close the current file and start a new file at that point. Therefore in usual operation, when a namenode starts up it will load an fsimage and requests to read transactions after that point, it will most likely find a file that starts from next transaction id.
        Alternatively, a file implementation can ignore mark and close a file every 100000 transactions. Now if it has to seek to 50000th transaction it can just read and ignore previous transactions. Since transaction files will be read only for checkpointing or at namenode startup or by backup at failover, it is not very expensive. In a recent measurement we found that namenode could load 1.4M transactions in 27 seconds.

        Also if we store edit logs to book keeper, 2NN can read from book keeper and there won't be a need for edit transfer, that is another attraction for using book keeper.

        > This seems like a somewhat serious flaw. If we anticipate using BK for HA..
        Agreed that the backup will lag behind the primary but when failover happens it can quickly read the additional transactions before declaring itself active. Won't that be an acceptable delay? There is some discussion on this in ZOOKEEPER-1016.

        > Another way of doing this is to say that, if an implementation does have this limitation, it can choose to "mark" whenever it likes.
        That is correct, however mark will be useful in the interface to be called before a savenamespace.

        > Most operations write the edit to the log while holding the FSN lock (to ensure serialized order between ops) and then drop the FSN lock to sync
        Good catch! A sync method is needed in EditLogOutputStream to be called after releasing the lock.

        > edit log transfer right now is based around the concept of discrete files which can be entirely fetched, with an associated md5sum
        I think it should be File storage implemenation's responsibility to keep an md5sum with every file, therefore the safety check while transferring files can still be supported.
        This interface doesn't manage transfer of edit logs. It only talks about reading/writing the transactions from/to a storage. When 2NN wants to do a checkpoint, it will download the files from primary, it will then get an EditLogInputStream object using this interface for the edit log files, and read the transactions.
        For Book-keeper storage, transfer will not be required.

        > md5sum /data/

        {1..4}

        /dfs/name/current/
        If we use a system like Book-keeper, we won't have the ability to perform this sanity check anyway. For different file storages, this ability will continue to exist, because a) mark will be called for all journal instances at the same time, and b) even if file storage implementation closes file every 100000 transactions it will be consistent for all files.

        > Refer to the discussion on HDFS-1073 about this property.
        Sure, I will look at it.

        Show
        Jitendra Nath Pandey added a comment - > In the file-based storage there's no clean way to seek to a particular transaction ID A savenamespace will be preceded by a call to mark (like current roll). A file implementation can close the current file and start a new file at that point. Therefore in usual operation, when a namenode starts up it will load an fsimage and requests to read transactions after that point, it will most likely find a file that starts from next transaction id. Alternatively, a file implementation can ignore mark and close a file every 100000 transactions. Now if it has to seek to 50000th transaction it can just read and ignore previous transactions. Since transaction files will be read only for checkpointing or at namenode startup or by backup at failover, it is not very expensive. In a recent measurement we found that namenode could load 1.4M transactions in 27 seconds. Also if we store edit logs to book keeper, 2NN can read from book keeper and there won't be a need for edit transfer, that is another attraction for using book keeper. > This seems like a somewhat serious flaw. If we anticipate using BK for HA.. Agreed that the backup will lag behind the primary but when failover happens it can quickly read the additional transactions before declaring itself active. Won't that be an acceptable delay? There is some discussion on this in ZOOKEEPER-1016 . > Another way of doing this is to say that, if an implementation does have this limitation, it can choose to "mark" whenever it likes. That is correct, however mark will be useful in the interface to be called before a savenamespace. > Most operations write the edit to the log while holding the FSN lock (to ensure serialized order between ops) and then drop the FSN lock to sync Good catch! A sync method is needed in EditLogOutputStream to be called after releasing the lock. > edit log transfer right now is based around the concept of discrete files which can be entirely fetched, with an associated md5sum I think it should be File storage implemenation's responsibility to keep an md5sum with every file, therefore the safety check while transferring files can still be supported. This interface doesn't manage transfer of edit logs. It only talks about reading/writing the transactions from/to a storage. When 2NN wants to do a checkpoint, it will download the files from primary, it will then get an EditLogInputStream object using this interface for the edit log files, and read the transactions. For Book-keeper storage, transfer will not be required. > md5sum /data/ {1..4} /dfs/name/current/ If we use a system like Book-keeper, we won't have the ability to perform this sanity check anyway. For different file storages, this ability will continue to exist, because a) mark will be called for all journal instances at the same time, and b) even if file storage implementation closes file every 100000 transactions it will be consistent for all files. > Refer to the discussion on HDFS-1073 about this property. Sure, I will look at it.
        Hide
        dhruba borthakur added a comment -

        Nice document. can you pl describe the error situations in more detail:

        1. what happens when the write to the outputstream throws an expcetion? can the client continue to write new transactions to the stream?
        2. what happens when the InputStream.read() encounters a partial transaction?
        3. from the InputStream, is it possible to know how many transactions are available (without actually reading till the end of the stream)?

        Show
        dhruba borthakur added a comment - Nice document. can you pl describe the error situations in more detail: 1. what happens when the write to the outputstream throws an expcetion? can the client continue to write new transactions to the stream? 2. what happens when the InputStream.read() encounters a partial transaction? 3. from the InputStream, is it possible to know how many transactions are available (without actually reading till the end of the stream)?
        Hide
        Jitendra Nath Pandey added a comment -

        Updated document includes error scenarios and addresses some comments.

        Show
        Jitendra Nath Pandey added a comment - Updated document includes error scenarios and addresses some comments.
        Hide
        Sanjay Radia added a comment -

        Please add some text doc to motivate the
        Journal#getNumberOfTransactions(long 
sinceTxnId)
.
        From what I understand this method gives the # of transactions since tX in a particular
        storage dir.
        Would it be better instead to ask getHighestTxid()?

        Show
        Sanjay Radia added a comment - Please add some text doc to motivate the Journal#getNumberOfTransactions(long 
sinceTxnId)
. From what I understand this method gives the # of transactions since tX in a particular storage dir. Would it be better instead to ask getHighestTxid()?
        Hide
        Ivan Kelly added a comment -

        Would it be better instead to ask getHighestTxid()?

        #getNumberOfTransactions() is useful for deciding which journal to use. Presumably sinceTxnId refers to the last transaction Id loaded from the image. So sinceTxnId + getNumberOfTransactions() gives you the maximum prefix of the NameNode history a particular journal can give you.

        #getHighestTxid() would, however, give you the highest transaction id without taking into consideration that gaps may exist.

        Show
        Ivan Kelly added a comment - Would it be better instead to ask getHighestTxid()? #getNumberOfTransactions() is useful for deciding which journal to use. Presumably sinceTxnId refers to the last transaction Id loaded from the image. So sinceTxnId + getNumberOfTransactions() gives you the maximum prefix of the NameNode history a particular journal can give you. #getHighestTxid() would, however, give you the highest transaction id without taking into consideration that gaps may exist.
        Hide
        Ivan Kelly added a comment -

        I've been working on implementing the design for HDFS-1580 on top of the
        HDFS-1073 branch and have run into a problem with
        #getNumberOfTransactions().

        Specifically, I've been working on the input code in FSImage:

           protected boolean loadEdits(JournalManager journal) throws IOException {
             LOG.debug("About to load edits:\n  " + journal);
        
             FSEditLogLoader loader = new FSEditLogLoader(namesystem);
             long startingTxId = storage.getMostRecentCheckpointTxId() + 1;
             int numLoaded = 0;
             // Load latest edits
        
             long numTransactionsToLoad = 
        journal.getNumberOfTransactions(startingTxId);
        
             while (numLoaded < numTransactionsToLoad) {
               EditLogInputStream editIn = journal.getInputStream(startingTxId);
               LOG.debug("Reading " + editIn + " expecting start txid #" + 
        startingTxId);
        
               int thisNumLoaded = loader.loadFSEdits(editIn, startingTxId);
        
               startingTxId += thisNumLoaded;
               numLoaded += thisNumLoaded;
               editIn.close();
             }
        
             // update the counts
             getFSNamesystem().dir.updateCountForINodeWithQuota();
        
             // update the txid for the edit log
             editLog.setNextTxId(storage.getMostRecentCheckpointTxId() + 
        numLoaded + 1);
        
             // If we loaded any edits, need to save.
             return numLoaded > 0;
           }
        

        The load is in a loop now, as the output is still in LogSegment form,
        but even in a single stream implementation getNumberOfTransactions()
        presents a problem.

        The problem is that sometimes it is impossible to return a number for
        getNumberOfTransactions(). This case is when NameNode has crashed in the
        middle of an edit log. The editlog is named edits_inprogress_N where N
        is the first transaction id in the edit log. But since NN crashed, we
        dont know the last transaction so we cannot possibly return the number
        of transactions in the journal without scanning the file from the start.

        Without getNumberOfTransactions() its difficult to choose which journal
        has the most edits. HDFS-1073 uses the number of bytes in the file, but
        this doesn't feel very safe for anything that isn't a file. Whats more,
        if the start transaction of two journal snippets are out of sync, then
        it becomes impossible to choose which journal has the most transactions
        using just filesize(This is an argument for log segments).

        The simplest solution I see is to actually scan the _inprogress file
        from the start to get the last transaction written. As this should only
        happen in NameNode crashes, the delay for doing this shouldn't be
        prohibitive.

        Show
        Ivan Kelly added a comment - I've been working on implementing the design for HDFS-1580 on top of the HDFS-1073 branch and have run into a problem with #getNumberOfTransactions(). Specifically, I've been working on the input code in FSImage: protected boolean loadEdits(JournalManager journal) throws IOException { LOG.debug( "About to load edits:\n " + journal); FSEditLogLoader loader = new FSEditLogLoader(namesystem); long startingTxId = storage.getMostRecentCheckpointTxId() + 1; int numLoaded = 0; // Load latest edits long numTransactionsToLoad = journal.getNumberOfTransactions(startingTxId); while (numLoaded < numTransactionsToLoad) { EditLogInputStream editIn = journal.getInputStream(startingTxId); LOG.debug( "Reading " + editIn + " expecting start txid #" + startingTxId); int thisNumLoaded = loader.loadFSEdits(editIn, startingTxId); startingTxId += thisNumLoaded; numLoaded += thisNumLoaded; editIn.close(); } // update the counts getFSNamesystem().dir.updateCountForINodeWithQuota(); // update the txid for the edit log editLog.setNextTxId(storage.getMostRecentCheckpointTxId() + numLoaded + 1); // If we loaded any edits, need to save. return numLoaded > 0; } The load is in a loop now, as the output is still in LogSegment form, but even in a single stream implementation getNumberOfTransactions() presents a problem. The problem is that sometimes it is impossible to return a number for getNumberOfTransactions(). This case is when NameNode has crashed in the middle of an edit log. The editlog is named edits_inprogress_N where N is the first transaction id in the edit log. But since NN crashed, we dont know the last transaction so we cannot possibly return the number of transactions in the journal without scanning the file from the start. Without getNumberOfTransactions() its difficult to choose which journal has the most edits. HDFS-1073 uses the number of bytes in the file, but this doesn't feel very safe for anything that isn't a file. Whats more, if the start transaction of two journal snippets are out of sync, then it becomes impossible to choose which journal has the most transactions using just filesize(This is an argument for log segments). The simplest solution I see is to actually scan the _inprogress file from the start to get the last transaction written. As this should only happen in NameNode crashes, the delay for doing this shouldn't be prohibitive.
        Hide
        Ivan Kelly added a comment -

        I've created HDFS-2003 to split FSEditLogLoader into reading and memory state building. This will make it easier to count the number of transactions in a journal.

        Show
        Ivan Kelly added a comment - I've created HDFS-2003 to split FSEditLogLoader into reading and memory state building. This will make it easier to count the number of transactions in a journal.
        Hide
        Todd Lipcon added a comment -

        Hey Ivan - yes, I agree that using the number of bytes is a bit lame, and it would be much nicer to be able to "recover" the inprogress files and properly rename them to an edits_N-M type "finalized" file at startup time. I think in fact there is a related bug which I commented about here

        I'll take a look at HDFS-2003

        Show
        Todd Lipcon added a comment - Hey Ivan - yes, I agree that using the number of bytes is a bit lame, and it would be much nicer to be able to "recover" the inprogress files and properly rename them to an edits_N-M type "finalized" file at startup time. I think in fact there is a related bug which I commented about here I'll take a look at HDFS-2003
        Hide
        Sanjay Radia added a comment -

        Finalizing the edits at initialization makes sense. It would require reading the
        edits but this should not be bad if we roll the edits at reasonable intervals.
        One possibility is for this to be part of init() that returns the highest txId.
        Jitendra had mentioned to me why he preferred the getNumTransaction(sinceTx) but I forget the
        reason.

        Show
        Sanjay Radia added a comment - Finalizing the edits at initialization makes sense. It would require reading the edits but this should not be bad if we roll the edits at reasonable intervals. One possibility is for this to be part of init() that returns the highest txId. Jitendra had mentioned to me why he preferred the getNumTransaction(sinceTx) but I forget the reason.
        Hide
        Ivan Kelly added a comment -

        Finalizing in getNumTransactions is a bit messy, because its difficult to tell if a inprogress file is actually inprogress or the result of a crashed namenode. I tried it in HDFS-2018, but ended up creating an explicit recoverUnclosedStreams() call, which only gets called on loading (getNumTransactions is called in other places, such as checkpointing). It would be possible to hide this inside the JournalManager, but then we'd have to make some sort of locking mechanism to ensure an inprogress stream doesn't get finalised while it is writing. This is simple for bookkeeper, but for filesystems we'd need some sort of lease mechanism.

        Show
        Ivan Kelly added a comment - Finalizing in getNumTransactions is a bit messy, because its difficult to tell if a inprogress file is actually inprogress or the result of a crashed namenode. I tried it in HDFS-2018 , but ended up creating an explicit recoverUnclosedStreams() call, which only gets called on loading (getNumTransactions is called in other places, such as checkpointing). It would be possible to hide this inside the JournalManager, but then we'd have to make some sort of locking mechanism to ensure an inprogress stream doesn't get finalised while it is writing. This is simple for bookkeeper, but for filesystems we'd need some sort of lease mechanism.
        Hide
        Todd Lipcon added a comment -

        I think we need that kind of "recover" interface anyway to do IO fencing on shared storage, when it comes to HA.

        Show
        Todd Lipcon added a comment - I think we need that kind of "recover" interface anyway to do IO fencing on shared storage, when it comes to HA.
        Hide
        Jitendra Nath Pandey added a comment -

        @Sanjay
        >Jitendra had mentioned to me why he preferred the getNumTransaction(sinceTx) but I >forget the reason.
        getNumTransaction(sinceTx) will throw an exception if it sees a gap (in sequence of transactions due to an earlier failure of the journal) after sinceTx. It will return a number only if it the journal can actually serve those many transactions starting from sinceTx.

        @Ivan
        >Finalizing in getNumTransactions is a bit messy.
        getNumTransactions will also be called by readers of edit logs. Finalize or recover should happen only in the context of the writer. I think finalization might make sense at the creation of output stream. For example, finalize the edit logs when namenode comes back up, after a crash, and opens output stream for writing. A separate recover method in the interface may also be useful.
        Two distinct cases where getNumTransactions can be used:
        (a) At namenode startup or backup at failover:
        In this case the in_progress file must be read to capture all the transactions. This is in the context of the writer.
        (b) Checkpointer, backup (non-failover case) or any other reader:
        In this case in_progress file can be ignored and checkpoint only up to the last rolled/finalized edit log file. This is the context of a reader.

        I think we have following options
        1) getNumTransactions reads in_progress file in both cases up to whatever can be read successfully. Caveat: Should checkpointer download the in_progress file as well?
        2) Don't read in_progress file, and handle case (a) by first calling a 'recover' method that finalizes the edit logs, and handle case (b) by rolling the edit logs.
        3) Third option, is to have two separate methods one that counts in_progress file and other doesn't.

        It seems to me option (1) is the simplest. Checkpointer doesn't need to download in_progress file, however for shared nfs storage it can read in_progress file too.

        Show
        Jitendra Nath Pandey added a comment - @Sanjay >Jitendra had mentioned to me why he preferred the getNumTransaction(sinceTx) but I >forget the reason. getNumTransaction(sinceTx) will throw an exception if it sees a gap (in sequence of transactions due to an earlier failure of the journal) after sinceTx. It will return a number only if it the journal can actually serve those many transactions starting from sinceTx. @Ivan >Finalizing in getNumTransactions is a bit messy. getNumTransactions will also be called by readers of edit logs. Finalize or recover should happen only in the context of the writer. I think finalization might make sense at the creation of output stream. For example, finalize the edit logs when namenode comes back up, after a crash, and opens output stream for writing. A separate recover method in the interface may also be useful. Two distinct cases where getNumTransactions can be used: (a) At namenode startup or backup at failover: In this case the in_progress file must be read to capture all the transactions. This is in the context of the writer. (b) Checkpointer, backup (non-failover case) or any other reader: In this case in_progress file can be ignored and checkpoint only up to the last rolled/finalized edit log file. This is the context of a reader. I think we have following options 1) getNumTransactions reads in_progress file in both cases up to whatever can be read successfully. Caveat: Should checkpointer download the in_progress file as well? 2) Don't read in_progress file, and handle case (a) by first calling a 'recover' method that finalizes the edit logs, and handle case (b) by rolling the edit logs. 3) Third option, is to have two separate methods one that counts in_progress file and other doesn't. It seems to me option (1) is the simplest. Checkpointer doesn't need to download in_progress file, however for shared nfs storage it can read in_progress file too.
        Hide
        Ivan Kelly added a comment -

        @Jitendra
        (1) should work for checkpointing, as if you journal A has more edits than journal B while counting the in_progress file, it will have more or an equal number not counting the in_progress file. More in the case that B has gaps in which case it throws an exception, equal otherwise.

        So we finalise inprogress when we open a write and spot an inprogress file. I guess this should only happen on startup after a crash. The writer shouldn't finalise an inprogress if something else is writing to it. We have nothing to prevent this now, but if this is happening, your system is broken. Fencing could be implemented later to explicitly exclude this possibility.

        Show
        Ivan Kelly added a comment - @Jitendra (1) should work for checkpointing, as if you journal A has more edits than journal B while counting the in_progress file, it will have more or an equal number not counting the in_progress file. More in the case that B has gaps in which case it throws an exception, equal otherwise. So we finalise inprogress when we open a write and spot an inprogress file. I guess this should only happen on startup after a crash. The writer shouldn't finalise an inprogress if something else is writing to it. We have nothing to prevent this now, but if this is happening, your system is broken. Fencing could be implemented later to explicitly exclude this possibility.
        Hide
        Jitendra Nath Pandey added a comment -

        The latest design document introduces JournalSet which will manage all the journals for the namenode. It is proposed to be an implementation of Journal interface only.

        Show
        Jitendra Nath Pandey added a comment - The latest design document introduces JournalSet which will manage all the journals for the namenode. It is proposed to be an implementation of Journal interface only.
        Hide
        dhruba borthakur added a comment -

        1. it would be nice to get rid of mark() method
        2. possibly rename InputStream.readNext() to InputStream.seekNext() just to make it clear that we are operating on some sort of an underlying stream

        Show
        dhruba borthakur added a comment - 1. it would be nice to get rid of mark() method 2. possibly rename InputStream.readNext() to InputStream.seekNext() just to make it clear that we are operating on some sort of an underlying stream
        Hide
        Ivan Kelly added a comment -

        @dhruba
        1. I agree. See 21st March comments on this JIRA.

        Show
        Ivan Kelly added a comment - @dhruba 1. I agree. See 21st March comments on this JIRA.
        Hide
        Ivan Kelly added a comment -

        This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper)

        There's 2 changes:
        1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode.

        2) Pluggable creation of journal managers.
        When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI).

        Show
        Ivan Kelly added a comment - This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper) There's 2 changes: 1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode. 2) Pluggable creation of journal managers. When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI).
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12500207/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        -1 javadoc. The javadoc tool appears to have generated 1 warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hdfs.TestDistributedFileSystem
        org.apache.hadoop.hdfs.server.datanode.TestDeleteBlockPool
        org.apache.hadoop.hdfs.TestAbandonBlock
        org.apache.hadoop.hdfs.server.namenode.TestBackupNode
        org.apache.hadoop.hdfs.tools.offlineEditsViewer.TestOfflineEditsViewer
        org.apache.hadoop.hdfs.TestRestartDFS

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1406//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1406//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12500207/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. -1 javadoc. The javadoc tool appears to have generated 1 warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed these unit tests: org.apache.hadoop.hdfs.TestDistributedFileSystem org.apache.hadoop.hdfs.server.datanode.TestDeleteBlockPool org.apache.hadoop.hdfs.TestAbandonBlock org.apache.hadoop.hdfs.server.namenode.TestBackupNode org.apache.hadoop.hdfs.tools.offlineEditsViewer.TestOfflineEditsViewer org.apache.hadoop.hdfs.TestRestartDFS +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1406//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1406//console This message is automatically generated.
        Hide
        Ivan Kelly added a comment -

        Removed noop opcode, which I had introduced for testing another thing. Shouldn't have been in this patch.

        Show
        Ivan Kelly added a comment - Removed noop opcode, which I had introduced for testing another thing. Shouldn't have been in this patch.
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12500367/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        -1 javadoc. The javadoc tool appears to have generated 1 warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hdfs.server.namenode.TestBackupNode
        org.apache.hadoop.hdfs.TestFileAppend2
        org.apache.hadoop.hdfs.TestBalancerBandwidth
        org.apache.hadoop.hdfs.TestRestartDFS
        org.apache.hadoop.hdfs.TestDistributedFileSystem
        org.apache.hadoop.hdfs.server.datanode.TestDeleteBlockPool

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1422//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1422//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12500367/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. -1 javadoc. The javadoc tool appears to have generated 1 warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed these unit tests: org.apache.hadoop.hdfs.server.namenode.TestBackupNode org.apache.hadoop.hdfs.TestFileAppend2 org.apache.hadoop.hdfs.TestBalancerBandwidth org.apache.hadoop.hdfs.TestRestartDFS org.apache.hadoop.hdfs.TestDistributedFileSystem org.apache.hadoop.hdfs.server.datanode.TestDeleteBlockPool +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1422//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1422//console This message is automatically generated.
        Hide
        Ivan Kelly added a comment -

        Updated patch to trunk

        Show
        Ivan Kelly added a comment - Updated patch to trunk
        Hide
        jiraposter@reviews.apache.org added a comment -

        -----------------------------------------------------------
        This is an automatically generated e-mail. To reply, visit:
        https://reviews.apache.org/r/2672/
        -----------------------------------------------------------

        Review request for hadoop-hdfs.

        Summary
        -------

        This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper)

        There's 2 changes:
        1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode.

        2) Pluggable creation of journal managers.
        When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI).

        This addresses bug HDFS-1580.
        http://issues.apache.org/jira/browse/HDFS-1580

        Diffs


        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java dd39676
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java 974697d
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java 067990d
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java 9db7f8a
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java 4780d04
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java c6f8505
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java 8681837
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java f80f863
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java 991fd08
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java 3adb439
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java 348e3ef
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java 45b5714
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java a7fa7fb
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java 4d7cfd8
        hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java PRE-CREATION

        Diff: https://reviews.apache.org/r/2672/diff

        Testing
        -------

        Thanks,

        Ivan

        Show
        jiraposter@reviews.apache.org added a comment - ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/2672/ ----------------------------------------------------------- Review request for hadoop-hdfs. Summary ------- This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper) There's 2 changes: 1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode. 2) Pluggable creation of journal managers. When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI). This addresses bug HDFS-1580 . http://issues.apache.org/jira/browse/HDFS-1580 Diffs hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java dd39676 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java 974697d hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java 067990d hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java 9db7f8a hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java 4780d04 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java c6f8505 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java 8681837 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java f80f863 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java 991fd08 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java 3adb439 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java 348e3ef hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java 45b5714 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java a7fa7fb hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java 4d7cfd8 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java PRE-CREATION Diff: https://reviews.apache.org/r/2672/diff Testing ------- Thanks, Ivan
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12501965/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        -1 javadoc. The javadoc tool appears to have generated 1 warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hdfs.TestFileAppend2
        org.apache.hadoop.hdfs.server.datanode.TestMulitipleNNDataBlockScanner
        org.apache.hadoop.hdfs.TestBalancerBandwidth

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1519//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1519//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12501965/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. -1 javadoc. The javadoc tool appears to have generated 1 warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed these unit tests: org.apache.hadoop.hdfs.TestFileAppend2 org.apache.hadoop.hdfs.server.datanode.TestMulitipleNNDataBlockScanner org.apache.hadoop.hdfs.TestBalancerBandwidth +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1519//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1519//console This message is automatically generated.
        Hide
        jiraposter@reviews.apache.org added a comment -

        -----------------------------------------------------------
        This is an automatically generated e-mail. To reply, visit:
        https://reviews.apache.org/r/2672/#review3014
        -----------------------------------------------------------

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        <https://reviews.apache.org/r/2672/#comment6727>

        we use _PREFIX instead of _BASE elsewhere for key prefixes

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        <https://reviews.apache.org/r/2672/#comment6728>

        why not just use conf.getClass here and return a Class? And throw the exception right here instead of returning null and throwing below

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        <https://reviews.apache.org/r/2672/#comment6729>

        this is the wrong layer - better to filter for file:// URLs where this is called, I think.

        hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        <https://reviews.apache.org/r/2672/#comment6730>

        no need to have any datanodes for any of these tests - will run faster without.

        hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        <https://reviews.apache.org/r/2672/#comment6731>

        our convention is to use american spelling (initialized)

        hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        <https://reviews.apache.org/r/2672/#comment6732>

        our style is to not have multiple classes per .java file unless they're inner classes. You can make this a static inner class of the test.

        hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        <https://reviews.apache.org/r/2672/#comment6733>

        just return Mockito.mock(EditLogOutputStream.class) and you don't need to have the whole implementation below

        • Todd

        On 2011-11-02 14:33:47, Ivan Kelly wrote:

        -----------------------------------------------------------

        This is an automatically generated e-mail. To reply, visit:

        https://reviews.apache.org/r/2672/

        -----------------------------------------------------------

        (Updated 2011-11-02 14:33:47)

        Review request for hadoop-hdfs.

        Summary

        -------

        This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper)

        There's 2 changes:

        1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode.

        2) Pluggable creation of journal managers.

        When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI).

        This addresses bug HDFS-1580.

        http://issues.apache.org/jira/browse/HDFS-1580

        Diffs

        -----

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java dd39676

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java 974697d

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java 067990d

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java 9db7f8a

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java 4780d04

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java c6f8505

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java 8681837

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java f80f863

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java 991fd08

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java 3adb439

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java 348e3ef

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java 45b5714

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java a7fa7fb

        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java 4d7cfd8

        hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java PRE-CREATION

        Diff: https://reviews.apache.org/r/2672/diff

        Testing

        -------

        Thanks,

        Ivan

        Show
        jiraposter@reviews.apache.org added a comment - ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/2672/#review3014 ----------------------------------------------------------- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java < https://reviews.apache.org/r/2672/#comment6727 > we use _PREFIX instead of _BASE elsewhere for key prefixes hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java < https://reviews.apache.org/r/2672/#comment6728 > why not just use conf.getClass here and return a Class? And throw the exception right here instead of returning null and throwing below hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java < https://reviews.apache.org/r/2672/#comment6729 > this is the wrong layer - better to filter for file:// URLs where this is called, I think. hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java < https://reviews.apache.org/r/2672/#comment6730 > no need to have any datanodes for any of these tests - will run faster without. hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java < https://reviews.apache.org/r/2672/#comment6731 > our convention is to use american spelling (initialized) hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java < https://reviews.apache.org/r/2672/#comment6732 > our style is to not have multiple classes per .java file unless they're inner classes. You can make this a static inner class of the test. hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java < https://reviews.apache.org/r/2672/#comment6733 > just return Mockito.mock(EditLogOutputStream.class) and you don't need to have the whole implementation below Todd On 2011-11-02 14:33:47, Ivan Kelly wrote: ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/2672/ ----------------------------------------------------------- (Updated 2011-11-02 14:33:47) Review request for hadoop-hdfs. Summary ------- This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper) There's 2 changes: 1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode. 2) Pluggable creation of journal managers. When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI). This addresses bug HDFS-1580 . http://issues.apache.org/jira/browse/HDFS-1580 Diffs ----- hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java dd39676 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java 974697d hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java 067990d hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java 9db7f8a hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java 4780d04 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java c6f8505 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java 8681837 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java f80f863 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java 991fd08 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java 3adb439 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java 348e3ef hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java 45b5714 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java a7fa7fb hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java 4d7cfd8 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java PRE-CREATION Diff: https://reviews.apache.org/r/2672/diff Testing ------- Thanks, Ivan
        Hide
        Ivan Kelly added a comment -

        Thanks for the review Todd. I've uploaded a new patch which addresses your comments.

        Show
        Ivan Kelly added a comment - Thanks for the review Todd. I've uploaded a new patch which addresses your comments.
        Hide
        jiraposter@reviews.apache.org added a comment -

        -----------------------------------------------------------
        This is an automatically generated e-mail. To reply, visit:
        https://reviews.apache.org/r/2672/
        -----------------------------------------------------------

        (Updated 2011-11-02 20:54:27.126053)

        Review request for hadoop-hdfs.

        Summary
        -------

        This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper)

        There's 2 changes:
        1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode.

        2) Pluggable creation of journal managers.
        When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI).

        This addresses bug HDFS-1580.
        http://issues.apache.org/jira/browse/HDFS-1580

        Diffs (updated)


        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java dd39676
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java 974697d
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java 067990d
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java 9db7f8a
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java 4780d04
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java c6f8505
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java 8681837
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java f80f863
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java 991fd08
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java 3adb439
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java 348e3ef
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java 45b5714
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java a7fa7fb
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java 4d7cfd8
        hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java PRE-CREATION

        Diff: https://reviews.apache.org/r/2672/diff

        Testing
        -------

        Thanks,

        Ivan

        Show
        jiraposter@reviews.apache.org added a comment - ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/2672/ ----------------------------------------------------------- (Updated 2011-11-02 20:54:27.126053) Review request for hadoop-hdfs. Summary ------- This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper) There's 2 changes: 1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode. 2) Pluggable creation of journal managers. When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI). This addresses bug HDFS-1580 . http://issues.apache.org/jira/browse/HDFS-1580 Diffs (updated) hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java dd39676 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java 974697d hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java 067990d hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java 9db7f8a hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java 4780d04 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java c6f8505 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java 8681837 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java f80f863 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java 991fd08 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java 3adb439 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java 348e3ef hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java 45b5714 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java a7fa7fb hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java 4d7cfd8 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java PRE-CREATION Diff: https://reviews.apache.org/r/2672/diff Testing ------- Thanks, Ivan
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12502029/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        -1 javadoc. The javadoc tool appears to have generated 1 warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS
        org.apache.hadoop.hdfs.server.namenode.TestListCorruptFileBlocks
        org.apache.hadoop.hdfs.TestFileCreationClient
        org.apache.hadoop.hdfs.server.blockmanagement.TestBlocksWithNotEnoughRacks
        org.apache.hadoop.hdfs.TestSetrepIncreasing

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1525//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1525//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12502029/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. -1 javadoc. The javadoc tool appears to have generated 1 warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed these unit tests: org.apache.hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS org.apache.hadoop.hdfs.server.namenode.TestListCorruptFileBlocks org.apache.hadoop.hdfs.TestFileCreationClient org.apache.hadoop.hdfs.server.blockmanagement.TestBlocksWithNotEnoughRacks org.apache.hadoop.hdfs.TestSetrepIncreasing +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1525//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1525//console This message is automatically generated.
        Hide
        jiraposter@reviews.apache.org added a comment -

        -----------------------------------------------------------
        This is an automatically generated e-mail. To reply, visit:
        https://reviews.apache.org/r/2672/
        -----------------------------------------------------------

        (Updated 2011-11-03 11:25:09.526158)

        Review request for hadoop-hdfs.

        Summary
        -------

        This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper)

        There's 2 changes:
        1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode.

        2) Pluggable creation of journal managers.
        When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI).

        This addresses bug HDFS-1580.
        http://issues.apache.org/jira/browse/HDFS-1580

        Diffs (updated)


        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java 7630335
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java 974697d
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java 067990d
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java 9db7f8a
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java 4780d04
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java c6f8505
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java 8681837
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java f80f863
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java 991fd08
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java 3adb439
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java 348e3ef
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java 45b5714
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java a7fa7fb
        hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java 4d7cfd8
        hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java PRE-CREATION

        Diff: https://reviews.apache.org/r/2672/diff

        Testing
        -------

        Thanks,

        Ivan

        Show
        jiraposter@reviews.apache.org added a comment - ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/2672/ ----------------------------------------------------------- (Updated 2011-11-03 11:25:09.526158) Review request for hadoop-hdfs. Summary ------- This is the final piece to allow the loading of custom implementations of JournalManager. There is another change HDFS-2334 which adds closeable to JournalManager, but that may not be absolutely necessary for all journal types. (it is for bookkeeper) There's 2 changes: 1) I've changes the interfaces(JournalManager, EditLogInputStream & EditLogOutputStream) so that they can be implemented outside of the org.apache.hadoop.hdfs.server.namenode. 2) Pluggable creation of journal managers. When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir, and it encounters a URI with a schema different to "file" it loads the name of the implementing class from "dfs.namenode.edits.journal-plugin.<schema>". This class must implement JournalManager and have a constructor which takes (Configuration, URI). This addresses bug HDFS-1580 . http://issues.apache.org/jira/browse/HDFS-1580 Diffs (updated) hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java 7630335 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java 974697d hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java 067990d hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java 9db7f8a hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java 4780d04 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java c6f8505 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java 8681837 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java f80f863 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java 991fd08 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java 3adb439 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java 348e3ef hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java 45b5714 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java a7fa7fb hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java 4d7cfd8 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java PRE-CREATION Diff: https://reviews.apache.org/r/2672/diff Testing ------- Thanks, Ivan
        Hide
        Ivan Kelly added a comment -

        Fixed javadoc issue.

        Show
        Ivan Kelly added a comment - Fixed javadoc issue.
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12502129/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hdfs.TestAbandonBlock

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1530//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1530//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12502129/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed these unit tests: org.apache.hadoop.hdfs.TestAbandonBlock +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1530//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1530//console This message is automatically generated.
        Hide
        Ivan Kelly added a comment -

        Local reruns of TestAbandonBlock passes cleanly. Seems flakey in general, also failed on Build #1527, Build #1527, Build #1527 & Build #1511. I didn't look back any further.

        Show
        Ivan Kelly added a comment - Local reruns of TestAbandonBlock passes cleanly. Seems flakey in general, also failed on Build #1527, Build #1527, Build #1527 & Build #1511. I didn't look back any further.
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12505101/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hdfs.TestHFlush
        org.apache.hadoop.hdfs.TestClientReportBadBlock
        org.apache.hadoop.hdfs.TestDFSShell
        org.apache.hadoop.hdfs.tools.TestGetGroups
        org.apache.hadoop.hdfs.TestDFSClientRetries
        org.apache.hadoop.hdfs.TestModTime
        org.apache.hadoop.fs.viewfs.TestViewFsHdfs
        org.apache.hadoop.hdfs.TestPread
        org.apache.hadoop.hdfs.web.TestFSMainOperationsWebHdfs
        org.apache.hadoop.hdfs.TestFileConcurrentReader

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1608//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1608//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12505101/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed these unit tests: org.apache.hadoop.hdfs.TestHFlush org.apache.hadoop.hdfs.TestClientReportBadBlock org.apache.hadoop.hdfs.TestDFSShell org.apache.hadoop.hdfs.tools.TestGetGroups org.apache.hadoop.hdfs.TestDFSClientRetries org.apache.hadoop.hdfs.TestModTime org.apache.hadoop.fs.viewfs.TestViewFsHdfs org.apache.hadoop.hdfs.TestPread org.apache.hadoop.hdfs.web.TestFSMainOperationsWebHdfs org.apache.hadoop.hdfs.TestFileConcurrentReader +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1608//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1608//console This message is automatically generated.
        Hide
        Jitendra Nath Pandey added a comment -
        1. Please annotate the public classes with InterfaceAudience.Private and also for stability.
        2. Instead of using "file" for the file scheme, please declare it as a constant.
        Show
        Jitendra Nath Pandey added a comment - Please annotate the public classes with InterfaceAudience.Private and also for stability. Instead of using "file" for the file scheme, please declare it as a constant.
        Hide
        Ivan Kelly added a comment -

        Thanks for reviewing Jitendra. I addressed comments in the latest patch. I also made the interfaces @InterfaceStability.Evolving
        as they have been changing lately and could still change a bit more with HA (if they haven't already).

        Show
        Ivan Kelly added a comment - Thanks for reviewing Jitendra. I addressed comments in the latest patch. I also made the interfaces @InterfaceStability.Evolving as they have been changing lately and could still change a bit more with HA (if they haven't already).
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12505753/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hdfs.TestHDFSServerPorts
        org.apache.hadoop.hdfs.TestBlocksScheduledCounter
        org.apache.hadoop.hdfs.TestAppendDifferentChecksum

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1622//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1622//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12505753/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed these unit tests: org.apache.hadoop.hdfs.TestHDFSServerPorts org.apache.hadoop.hdfs.TestBlocksScheduledCounter org.apache.hadoop.hdfs.TestAppendDifferentChecksum +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1622//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1622//console This message is automatically generated.
        Hide
        Todd Lipcon added a comment -

        Looks like TestDecommission timed out... is that timing out on trunk?

        Show
        Todd Lipcon added a comment - Looks like TestDecommission timed out... is that timing out on trunk?
        Hide
        Ivan Kelly added a comment -

        I couldn't find any, but multiple reruns of it and the surrounding tests didnt have any fails :/

        Running org.apache.hadoop.hdfs.TestMissingBlocksAlert
        Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 7.422 sec
        Running org.apache.hadoop.hdfs.TestDecommission
        Tests run: 7, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 172.954 sec
        Running org.apache.hadoop.hdfs.TestAppendDifferentChecksum
        Tests run: 3, Failures: 0, Errors: 0, Skipped: 1, Time elapsed: 7.52 sec
        Running org.apache.hadoop.hdfs.TestBlocksScheduledCounter
        Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 2.017 sec
        Running org.apache.hadoop.hdfs.server.namenode.TestGenericJournalConf
        Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 5.41 sec
        

        Im going to reupload the patch to kick off tests again.

        Show
        Ivan Kelly added a comment - I couldn't find any, but multiple reruns of it and the surrounding tests didnt have any fails :/ Running org.apache.hadoop.hdfs.TestMissingBlocksAlert Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 7.422 sec Running org.apache.hadoop.hdfs.TestDecommission Tests run: 7, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 172.954 sec Running org.apache.hadoop.hdfs.TestAppendDifferentChecksum Tests run: 3, Failures: 0, Errors: 0, Skipped: 1, Time elapsed: 7.52 sec Running org.apache.hadoop.hdfs.TestBlocksScheduledCounter Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 2.017 sec Running org.apache.hadoop.hdfs.server.namenode.TestGenericJournalConf Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 5.41 sec Im going to reupload the patch to kick off tests again.
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12505819/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hdfs.TestDistributedFileSystem
        org.apache.hadoop.hdfs.TestConnCache
        org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite
        org.apache.hadoop.hdfs.TestSafeMode
        org.apache.hadoop.hdfs.server.datanode.TestDataNodeExit
        org.apache.hadoop.hdfs.server.namenode.TestHDFSConcat
        org.apache.hadoop.hdfs.TestFileAppend
        org.apache.hadoop.hdfs.TestDatanodeDeath
        org.apache.hadoop.hdfs.server.namenode.TestFileLimit
        org.apache.hadoop.hdfs.TestLease
        org.apache.hadoop.hdfs.TestShortCircuitLocalRead
        org.apache.hadoop.hdfs.TestDFSRename
        org.apache.hadoop.hdfs.TestPipelines
        org.apache.hadoop.hdfs.TestSetrepDecreasing
        org.apache.hadoop.hdfs.TestLeaseRecovery2
        org.apache.hadoop.hdfs.TestFileAppend2
        org.apache.hadoop.security.TestRefreshUserMappings
        org.apache.hadoop.hdfs.server.datanode.TestRefreshNamenodes
        org.apache.hadoop.hdfs.TestHFlush
        org.apache.hadoop.hdfs.TestLeaseRecovery
        org.apache.hadoop.hdfs.web.TestWebHdfsFileSystemContract
        org.apache.hadoop.fs.loadGenerator.TestLoadGenerator
        org.apache.hadoop.hdfs.server.datanode.TestMulitipleNNDataBlockScanner
        org.apache.hadoop.hdfs.server.blockmanagement.TestNodeCount
        org.apache.hadoop.fs.TestGlobPaths
        org.apache.hadoop.hdfs.server.namenode.TestBlockUnderConstruction
        org.apache.hadoop.fs.TestResolveHdfsSymlink
        org.apache.hadoop.hdfs.TestFileCreationNamenodeRestart
        org.apache.hadoop.hdfs.server.namenode.TestListCorruptFileBlocks
        org.apache.hadoop.hdfs.server.namenode.TestCheckPointForSecurityTokens
        org.apache.hadoop.hdfs.TestDfsOverAvroRpc
        org.apache.hadoop.hdfs.server.blockmanagement.TestReplicationPolicy
        org.apache.hadoop.hdfs.server.datanode.TestDeleteBlockPool
        org.apache.hadoop.hdfs.server.namenode.TestStorageRestore
        org.apache.hadoop.hdfs.server.namenode.TestNameNodeResourceChecker
        org.apache.hadoop.hdfs.TestAbandonBlock
        org.apache.hadoop.fs.TestUrlStreamHandler
        org.apache.hadoop.hdfs.server.namenode.TestTransferFsImage
        org.apache.hadoop.hdfs.TestBlockMissingException
        org.apache.hadoop.fs.viewfs.TestViewFsFileStatusHdfs
        org.apache.hadoop.hdfs.TestClientReportBadBlock
        org.apache.hadoop.hdfs.TestDFSShell
        org.apache.hadoop.hdfs.server.datanode.TestDataNodeMultipleRegistrations
        org.apache.hadoop.hdfs.server.datanode.TestDataNodeVolumeFailure
        org.apache.hadoop.hdfs.server.namenode.TestProcessCorruptBlocks
        org.apache.hadoop.hdfs.TestFileCreation
        org.apache.hadoop.fs.TestFcHdfsPermission
        org.apache.hadoop.hdfs.TestCrcCorruption
        org.apache.hadoop.hdfs.server.namenode.TestNamenodeCapacityReport
        org.apache.hadoop.hdfs.TestListFilesInFileContext
        org.apache.hadoop.hdfs.server.datanode.TestBlockReplacement
        org.apache.hadoop.hdfs.server.namenode.TestNameEditsConfigs
        org.apache.hadoop.hdfs.TestListPathServlet
        org.apache.hadoop.hdfs.TestMultiThreadedHflush
        org.apache.hadoop.hdfs.TestDFSFinalize
        org.apache.hadoop.hdfs.TestLocalDFS
        org.apache.hadoop.hdfs.server.namenode.TestBackupNode
        org.apache.hadoop.hdfs.TestSeekBug
        org.apache.hadoop.hdfs.TestDFSStartupVersions
        org.apache.hadoop.hdfs.server.balancer.TestBalancer
        org.apache.hadoop.hdfs.tools.TestGetGroups
        org.apache.hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
        org.apache.hadoop.hdfs.server.namenode.metrics.TestNNMetricFilesInGetListingOps
        org.apache.hadoop.hdfs.TestDataTransferProtocol
        org.apache.hadoop.hdfs.TestDFSClientRetries
        org.apache.hadoop.hdfs.TestAppendDifferentChecksum
        org.apache.hadoop.hdfs.TestFileCreationDelete
        org.apache.hadoop.hdfs.tools.offlineEditsViewer.TestOfflineEditsViewer
        org.apache.hadoop.hdfs.server.common.TestDistributedUpgrade
        org.apache.hadoop.hdfs.TestMiniDFSCluster
        org.apache.hadoop.hdfs.server.namenode.TestLargeDirectoryDelete
        org.apache.hadoop.hdfs.TestFileCreationClient
        org.apache.hadoop.hdfs.server.datanode.TestWriteToReplica
        org.apache.hadoop.hdfs.TestListFilesInDFS
        org.apache.hadoop.hdfs.TestDatanodeConfig
        org.apache.hadoop.hdfs.server.namenode.TestNameNodeMXBean
        org.apache.hadoop.hdfs.server.namenode.TestMetaSave
        org.apache.hadoop.hdfs.server.namenode.TestFsck
        org.apache.hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks
        org.apache.hadoop.hdfs.TestDFSMkdirs
        org.apache.hadoop.hdfs.TestMissingBlocksAlert
        org.apache.hadoop.fs.TestFcHdfsCreateMkdir
        org.apache.hadoop.hdfs.server.namenode.TestAllowFormat
        org.apache.hadoop.hdfs.TestModTime
        org.apache.hadoop.hdfs.TestDFSRollback
        org.apache.hadoop.security.TestPermission
        org.apache.hadoop.hdfs.server.namenode.TestEditLogFileOutputStream
        org.apache.hadoop.hdfs.server.namenode.TestEditLog
        org.apache.hadoop.hdfs.server.namenode.TestStreamFile
        org.apache.hadoop.fs.viewfs.TestViewFileSystemHdfs
        org.apache.hadoop.hdfs.TestDecommission
        org.apache.hadoop.fs.TestHDFSFileContextMainOperations
        org.apache.hadoop.hdfs.server.namenode.TestDeadDatanode
        org.apache.hadoop.hdfs.server.datanode.TestDiskError
        org.apache.hadoop.hdfs.server.datanode.TestTransferRbw
        org.apache.hadoop.hdfs.server.namenode.TestSecondaryWebUi
        org.apache.hadoop.hdfs.server.datanode.TestDataNodeMXBean
        org.apache.hadoop.hdfs.TestDatanodeBlockScanner
        org.apache.hadoop.hdfs.TestFileCorruption
        org.apache.hadoop.hdfs.TestClientBlockVerification
        org.apache.hadoop.hdfs.server.namenode.TestSecurityTokenEditLog
        org.apache.hadoop.hdfs.TestClientProtocolForPipelineRecovery
        org.apache.hadoop.hdfs.TestFileAppend3
        org.apache.hadoop.hdfs.server.blockmanagement.TestOverReplicatedBlocks
        org.apache.hadoop.hdfs.TestFileAppend4
        org.apache.hadoop.hdfs.TestBlocksScheduledCounter
        org.apache.hadoop.hdfs.server.namenode.TestDecommissioningStatus
        org.apache.hadoop.hdfs.server.namenode.TestStartup
        org.apache.hadoop.hdfs.server.namenode.TestGenericJournalConf
        org.apache.hadoop.hdfs.server.datanode.TestDatanodeJsp
        org.apache.hadoop.hdfs.TestWriteRead
        org.apache.hadoop.hdfs.TestRenameWhileOpen
        org.apache.hadoop.hdfs.TestDFSUpgradeFromImage
        org.apache.hadoop.hdfs.TestDFSRemove
        org.apache.hadoop.fs.viewfs.TestViewFsHdfs
        org.apache.hadoop.hdfs.server.namenode.TestCheckpoint
        org.apache.hadoop.hdfs.TestDatanodeReport
        org.apache.hadoop.hdfs.TestHDFSFileSystemContract
        org.apache.hadoop.hdfs.TestDFSClientExcludedNodes
        org.apache.hadoop.hdfs.TestFSInputChecker
        org.apache.hadoop.hdfs.server.datanode.TestDataNodeMetrics
        org.apache.hadoop.hdfs.server.datanode.TestBlockReport
        org.apache.hadoop.hdfs.server.datanode.TestDirectoryScanner
        org.apache.hadoop.hdfs.TestSetTimes
        org.apache.hadoop.hdfs.TestPread
        org.apache.hadoop.hdfs.security.TestClientProtocolWithDelegationToken
        org.apache.hadoop.hdfs.server.namenode.TestEditLogRace
        org.apache.hadoop.hdfs.security.TestDelegationToken
        org.apache.hadoop.hdfs.server.namenode.TestNNStorageRetentionFunctional
        org.apache.hadoop.hdfs.TestDFSShellGenericOptions
        org.apache.hadoop.hdfs.TestDFSPermission
        org.apache.hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS
        org.apache.hadoop.hdfs.TestHDFSTrash
        org.apache.hadoop.hdfs.security.TestDelegationTokenForProxyUser
        org.apache.hadoop.fs.TestFcHdfsSymlink
        org.apache.hadoop.hdfs.server.namenode.TestNNThroughputBenchmark
        org.apache.hadoop.hdfs.TestFileStatus
        org.apache.hadoop.hdfs.TestDFSStorageStateRecovery
        org.apache.hadoop.hdfs.server.blockmanagement.TestHeartbeatHandling
        org.apache.hadoop.hdfs.server.namenode.TestNameNodeJspHelper
        org.apache.hadoop.hdfs.TestReadWhileWriting
        org.apache.hadoop.hdfs.TestHftpFileSystem
        org.apache.hadoop.hdfs.server.datanode.TestDataNodeVolumeFailureReporting
        org.apache.hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
        org.apache.hadoop.hdfs.server.namenode.TestSafeMode
        org.apache.hadoop.hdfs.TestDatanodeRegistration
        org.apache.hadoop.hdfs.server.namenode.TestCorruptFilesJsp
        org.apache.hadoop.hdfs.server.namenode.TestEditLogJournalFailures
        org.apache.hadoop.hdfs.TestDFSAddressConfig
        org.apache.hadoop.hdfs.TestLargeBlock
        org.apache.hadoop.hdfs.server.blockmanagement.TestComputeInvalidateWork
        org.apache.hadoop.hdfs.TestParallelRead
        org.apache.hadoop.hdfs.TestInjectionForSimulatedStorage
        org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol
        org.apache.hadoop.hdfs.server.datanode.TestDatanodeRestart
        org.apache.hadoop.hdfs.server.balancer.TestBalancerWithMultipleNameNodes
        org.apache.hadoop.hdfs.TestFSOutputSummer
        org.apache.hadoop.hdfs.TestFileCreationEmpty
        org.apache.hadoop.hdfs.TestBalancerBandwidth
        org.apache.hadoop.hdfs.TestWriteConfigurationToDFS
        org.apache.hadoop.fs.permission.TestStickyBit
        org.apache.hadoop.hdfs.server.namenode.TestFSEditLogLoader
        org.apache.hadoop.hdfs.security.token.block.TestBlockToken
        org.apache.hadoop.hdfs.TestRestartDFS
        org.apache.hadoop.hdfs.TestGetBlocks
        org.apache.hadoop.hdfs.TestSmallBlock
        org.apache.hadoop.hdfs.TestSetrepIncreasing
        org.apache.hadoop.hdfs.TestHDFSServerPorts
        org.apache.hadoop.hdfs.server.blockmanagement.TestBlocksWithNotEnoughRacks
        org.apache.hadoop.hdfs.server.namenode.TestValidateConfigurationSettings
        org.apache.hadoop.hdfs.TestReplication
        org.apache.hadoop.hdfs.TestDFSUpgrade
        org.apache.hadoop.hdfs.web.TestFSMainOperationsWebHdfs
        org.apache.hadoop.tools.TestJMXGet
        org.apache.hadoop.hdfs.TestReplaceDatanodeOnFailure
        org.apache.hadoop.hdfs.server.datanode.TestDataNodeVolumeFailureToleration
        org.apache.hadoop.hdfs.TestQuota
        org.apache.hadoop.hdfs.TestFileConcurrentReader

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1626//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1626//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12505819/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed these unit tests: org.apache.hadoop.hdfs.TestDistributedFileSystem org.apache.hadoop.hdfs.TestConnCache org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite org.apache.hadoop.hdfs.TestSafeMode org.apache.hadoop.hdfs.server.datanode.TestDataNodeExit org.apache.hadoop.hdfs.server.namenode.TestHDFSConcat org.apache.hadoop.hdfs.TestFileAppend org.apache.hadoop.hdfs.TestDatanodeDeath org.apache.hadoop.hdfs.server.namenode.TestFileLimit org.apache.hadoop.hdfs.TestLease org.apache.hadoop.hdfs.TestShortCircuitLocalRead org.apache.hadoop.hdfs.TestDFSRename org.apache.hadoop.hdfs.TestPipelines org.apache.hadoop.hdfs.TestSetrepDecreasing org.apache.hadoop.hdfs.TestLeaseRecovery2 org.apache.hadoop.hdfs.TestFileAppend2 org.apache.hadoop.security.TestRefreshUserMappings org.apache.hadoop.hdfs.server.datanode.TestRefreshNamenodes org.apache.hadoop.hdfs.TestHFlush org.apache.hadoop.hdfs.TestLeaseRecovery org.apache.hadoop.hdfs.web.TestWebHdfsFileSystemContract org.apache.hadoop.fs.loadGenerator.TestLoadGenerator org.apache.hadoop.hdfs.server.datanode.TestMulitipleNNDataBlockScanner org.apache.hadoop.hdfs.server.blockmanagement.TestNodeCount org.apache.hadoop.fs.TestGlobPaths org.apache.hadoop.hdfs.server.namenode.TestBlockUnderConstruction org.apache.hadoop.fs.TestResolveHdfsSymlink org.apache.hadoop.hdfs.TestFileCreationNamenodeRestart org.apache.hadoop.hdfs.server.namenode.TestListCorruptFileBlocks org.apache.hadoop.hdfs.server.namenode.TestCheckPointForSecurityTokens org.apache.hadoop.hdfs.TestDfsOverAvroRpc org.apache.hadoop.hdfs.server.blockmanagement.TestReplicationPolicy org.apache.hadoop.hdfs.server.datanode.TestDeleteBlockPool org.apache.hadoop.hdfs.server.namenode.TestStorageRestore org.apache.hadoop.hdfs.server.namenode.TestNameNodeResourceChecker org.apache.hadoop.hdfs.TestAbandonBlock org.apache.hadoop.fs.TestUrlStreamHandler org.apache.hadoop.hdfs.server.namenode.TestTransferFsImage org.apache.hadoop.hdfs.TestBlockMissingException org.apache.hadoop.fs.viewfs.TestViewFsFileStatusHdfs org.apache.hadoop.hdfs.TestClientReportBadBlock org.apache.hadoop.hdfs.TestDFSShell org.apache.hadoop.hdfs.server.datanode.TestDataNodeMultipleRegistrations org.apache.hadoop.hdfs.server.datanode.TestDataNodeVolumeFailure org.apache.hadoop.hdfs.server.namenode.TestProcessCorruptBlocks org.apache.hadoop.hdfs.TestFileCreation org.apache.hadoop.fs.TestFcHdfsPermission org.apache.hadoop.hdfs.TestCrcCorruption org.apache.hadoop.hdfs.server.namenode.TestNamenodeCapacityReport org.apache.hadoop.hdfs.TestListFilesInFileContext org.apache.hadoop.hdfs.server.datanode.TestBlockReplacement org.apache.hadoop.hdfs.server.namenode.TestNameEditsConfigs org.apache.hadoop.hdfs.TestListPathServlet org.apache.hadoop.hdfs.TestMultiThreadedHflush org.apache.hadoop.hdfs.TestDFSFinalize org.apache.hadoop.hdfs.TestLocalDFS org.apache.hadoop.hdfs.server.namenode.TestBackupNode org.apache.hadoop.hdfs.TestSeekBug org.apache.hadoop.hdfs.TestDFSStartupVersions org.apache.hadoop.hdfs.server.balancer.TestBalancer org.apache.hadoop.hdfs.tools.TestGetGroups org.apache.hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer org.apache.hadoop.hdfs.server.namenode.metrics.TestNNMetricFilesInGetListingOps org.apache.hadoop.hdfs.TestDataTransferProtocol org.apache.hadoop.hdfs.TestDFSClientRetries org.apache.hadoop.hdfs.TestAppendDifferentChecksum org.apache.hadoop.hdfs.TestFileCreationDelete org.apache.hadoop.hdfs.tools.offlineEditsViewer.TestOfflineEditsViewer org.apache.hadoop.hdfs.server.common.TestDistributedUpgrade org.apache.hadoop.hdfs.TestMiniDFSCluster org.apache.hadoop.hdfs.server.namenode.TestLargeDirectoryDelete org.apache.hadoop.hdfs.TestFileCreationClient org.apache.hadoop.hdfs.server.datanode.TestWriteToReplica org.apache.hadoop.hdfs.TestListFilesInDFS org.apache.hadoop.hdfs.TestDatanodeConfig org.apache.hadoop.hdfs.server.namenode.TestNameNodeMXBean org.apache.hadoop.hdfs.server.namenode.TestMetaSave org.apache.hadoop.hdfs.server.namenode.TestFsck org.apache.hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks org.apache.hadoop.hdfs.TestDFSMkdirs org.apache.hadoop.hdfs.TestMissingBlocksAlert org.apache.hadoop.fs.TestFcHdfsCreateMkdir org.apache.hadoop.hdfs.server.namenode.TestAllowFormat org.apache.hadoop.hdfs.TestModTime org.apache.hadoop.hdfs.TestDFSRollback org.apache.hadoop.security.TestPermission org.apache.hadoop.hdfs.server.namenode.TestEditLogFileOutputStream org.apache.hadoop.hdfs.server.namenode.TestEditLog org.apache.hadoop.hdfs.server.namenode.TestStreamFile org.apache.hadoop.fs.viewfs.TestViewFileSystemHdfs org.apache.hadoop.hdfs.TestDecommission org.apache.hadoop.fs.TestHDFSFileContextMainOperations org.apache.hadoop.hdfs.server.namenode.TestDeadDatanode org.apache.hadoop.hdfs.server.datanode.TestDiskError org.apache.hadoop.hdfs.server.datanode.TestTransferRbw org.apache.hadoop.hdfs.server.namenode.TestSecondaryWebUi org.apache.hadoop.hdfs.server.datanode.TestDataNodeMXBean org.apache.hadoop.hdfs.TestDatanodeBlockScanner org.apache.hadoop.hdfs.TestFileCorruption org.apache.hadoop.hdfs.TestClientBlockVerification org.apache.hadoop.hdfs.server.namenode.TestSecurityTokenEditLog org.apache.hadoop.hdfs.TestClientProtocolForPipelineRecovery org.apache.hadoop.hdfs.TestFileAppend3 org.apache.hadoop.hdfs.server.blockmanagement.TestOverReplicatedBlocks org.apache.hadoop.hdfs.TestFileAppend4 org.apache.hadoop.hdfs.TestBlocksScheduledCounter org.apache.hadoop.hdfs.server.namenode.TestDecommissioningStatus org.apache.hadoop.hdfs.server.namenode.TestStartup org.apache.hadoop.hdfs.server.namenode.TestGenericJournalConf org.apache.hadoop.hdfs.server.datanode.TestDatanodeJsp org.apache.hadoop.hdfs.TestWriteRead org.apache.hadoop.hdfs.TestRenameWhileOpen org.apache.hadoop.hdfs.TestDFSUpgradeFromImage org.apache.hadoop.hdfs.TestDFSRemove org.apache.hadoop.fs.viewfs.TestViewFsHdfs org.apache.hadoop.hdfs.server.namenode.TestCheckpoint org.apache.hadoop.hdfs.TestDatanodeReport org.apache.hadoop.hdfs.TestHDFSFileSystemContract org.apache.hadoop.hdfs.TestDFSClientExcludedNodes org.apache.hadoop.hdfs.TestFSInputChecker org.apache.hadoop.hdfs.server.datanode.TestDataNodeMetrics org.apache.hadoop.hdfs.server.datanode.TestBlockReport org.apache.hadoop.hdfs.server.datanode.TestDirectoryScanner org.apache.hadoop.hdfs.TestSetTimes org.apache.hadoop.hdfs.TestPread org.apache.hadoop.hdfs.security.TestClientProtocolWithDelegationToken org.apache.hadoop.hdfs.server.namenode.TestEditLogRace org.apache.hadoop.hdfs.security.TestDelegationToken org.apache.hadoop.hdfs.server.namenode.TestNNStorageRetentionFunctional org.apache.hadoop.hdfs.TestDFSShellGenericOptions org.apache.hadoop.hdfs.TestDFSPermission org.apache.hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS org.apache.hadoop.hdfs.TestHDFSTrash org.apache.hadoop.hdfs.security.TestDelegationTokenForProxyUser org.apache.hadoop.fs.TestFcHdfsSymlink org.apache.hadoop.hdfs.server.namenode.TestNNThroughputBenchmark org.apache.hadoop.hdfs.TestFileStatus org.apache.hadoop.hdfs.TestDFSStorageStateRecovery org.apache.hadoop.hdfs.server.blockmanagement.TestHeartbeatHandling org.apache.hadoop.hdfs.server.namenode.TestNameNodeJspHelper org.apache.hadoop.hdfs.TestReadWhileWriting org.apache.hadoop.hdfs.TestHftpFileSystem org.apache.hadoop.hdfs.server.datanode.TestDataNodeVolumeFailureReporting org.apache.hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics org.apache.hadoop.hdfs.server.namenode.TestSafeMode org.apache.hadoop.hdfs.TestDatanodeRegistration org.apache.hadoop.hdfs.server.namenode.TestCorruptFilesJsp org.apache.hadoop.hdfs.server.namenode.TestEditLogJournalFailures org.apache.hadoop.hdfs.TestDFSAddressConfig org.apache.hadoop.hdfs.TestLargeBlock org.apache.hadoop.hdfs.server.blockmanagement.TestComputeInvalidateWork org.apache.hadoop.hdfs.TestParallelRead org.apache.hadoop.hdfs.TestInjectionForSimulatedStorage org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol org.apache.hadoop.hdfs.server.datanode.TestDatanodeRestart org.apache.hadoop.hdfs.server.balancer.TestBalancerWithMultipleNameNodes org.apache.hadoop.hdfs.TestFSOutputSummer org.apache.hadoop.hdfs.TestFileCreationEmpty org.apache.hadoop.hdfs.TestBalancerBandwidth org.apache.hadoop.hdfs.TestWriteConfigurationToDFS org.apache.hadoop.fs.permission.TestStickyBit org.apache.hadoop.hdfs.server.namenode.TestFSEditLogLoader org.apache.hadoop.hdfs.security.token.block.TestBlockToken org.apache.hadoop.hdfs.TestRestartDFS org.apache.hadoop.hdfs.TestGetBlocks org.apache.hadoop.hdfs.TestSmallBlock org.apache.hadoop.hdfs.TestSetrepIncreasing org.apache.hadoop.hdfs.TestHDFSServerPorts org.apache.hadoop.hdfs.server.blockmanagement.TestBlocksWithNotEnoughRacks org.apache.hadoop.hdfs.server.namenode.TestValidateConfigurationSettings org.apache.hadoop.hdfs.TestReplication org.apache.hadoop.hdfs.TestDFSUpgrade org.apache.hadoop.hdfs.web.TestFSMainOperationsWebHdfs org.apache.hadoop.tools.TestJMXGet org.apache.hadoop.hdfs.TestReplaceDatanodeOnFailure org.apache.hadoop.hdfs.server.datanode.TestDataNodeVolumeFailureToleration org.apache.hadoop.hdfs.TestQuota org.apache.hadoop.hdfs.TestFileConcurrentReader +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1626//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1626//console This message is automatically generated.
        Hide
        Ivan Kelly added a comment -

        >:/ Something else seems to have side swiped the last build. None of the tests could run due to inability to create native threads. Kicking off again.

        Show
        Ivan Kelly added a comment - >:/ Something else seems to have side swiped the last build. None of the tests could run due to inability to create native threads. Kicking off again.
        Hide
        Hadoop QA added a comment -

        +1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12505861/HDFS-1580.diff
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 6 new or modified tests.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        +1 core tests. The patch passed unit tests in .

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1628//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1628//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - +1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12505861/HDFS-1580.diff against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. +1 core tests. The patch passed unit tests in . +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/1628//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/1628//console This message is automatically generated.
        Hide
        Jitendra Nath Pandey added a comment -

        +1. Patch looks good to me.

        Show
        Jitendra Nath Pandey added a comment - +1. Patch looks good to me.
        Hide
        Jitendra Nath Pandey added a comment -

        Committed to trunk. Thanks to Ivan.

        Show
        Jitendra Nath Pandey added a comment - Committed to trunk. Thanks to Ivan.
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Common-trunk-Commit #1366 (See https://builds.apache.org/job/Hadoop-Common-trunk-Commit/1366/)
        HDFS-1580. Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly.

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Show
        Hudson added a comment - Integrated in Hadoop-Common-trunk-Commit #1366 (See https://builds.apache.org/job/Hadoop-Common-trunk-Commit/1366/ ) HDFS-1580 . Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly. jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Mapreduce-trunk-Commit #1391 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk-Commit/1391/)
        HDFS-1580. Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly.

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Show
        Hudson added a comment - Integrated in Hadoop-Mapreduce-trunk-Commit #1391 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk-Commit/1391/ ) HDFS-1580 . Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly. jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Hdfs-trunk-Commit #1439 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk-Commit/1439/)
        HDFS-1580. Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly.

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Show
        Hudson added a comment - Integrated in Hadoop-Hdfs-trunk-Commit #1439 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk-Commit/1439/ ) HDFS-1580 . Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly. jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Hdfs-trunk-Commit #1443 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk-Commit/1443/)
        HDFS-1580. Committing the test case, it was missed in earlier commit.

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Show
        Hudson added a comment - Integrated in Hadoop-Hdfs-trunk-Commit #1443 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk-Commit/1443/ ) HDFS-1580 . Committing the test case, it was missed in earlier commit. jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Common-trunk-Commit #1371 (See https://builds.apache.org/job/Hadoop-Common-trunk-Commit/1371/)
        HDFS-1580. Committing the test case, it was missed in earlier commit.

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Show
        Hudson added a comment - Integrated in Hadoop-Common-trunk-Commit #1371 (See https://builds.apache.org/job/Hadoop-Common-trunk-Commit/1371/ ) HDFS-1580 . Committing the test case, it was missed in earlier commit. jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Mapreduce-trunk-Commit #1395 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk-Commit/1395/)
        HDFS-1580. Committing the test case, it was missed in earlier commit.

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Show
        Hudson added a comment - Integrated in Hadoop-Mapreduce-trunk-Commit #1395 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk-Commit/1395/ ) HDFS-1580 . Committing the test case, it was missed in earlier commit. jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Hdfs-trunk #886 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/886/)
        HDFS-1580. Committing the test case, it was missed in earlier commit.
        HDFS-1580. Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly.

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Show
        Hudson added a comment - Integrated in Hadoop-Hdfs-trunk #886 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/886/ ) HDFS-1580 . Committing the test case, it was missed in earlier commit. HDFS-1580 . Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly. jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Mapreduce-trunk #919 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/919/)
        HDFS-1580. Committing the test case, it was missed in earlier commit.
        HDFS-1580. Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly.

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java

        jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Show
        Hudson added a comment - Integrated in Hadoop-Mapreduce-trunk #919 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/919/ ) HDFS-1580 . Committing the test case, it was missed in earlier commit. HDFS-1580 . Add interface for generic Write Ahead Logging mechanisms. Contributed by Ivan Kelly. jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210749 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java jitendra : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1210602 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Hdfs-0.23-Commit #645 (See https://builds.apache.org/job/Hadoop-Hdfs-0.23-Commit/645/)
        HDFS-1580. Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864)

        Result = SUCCESS
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864
        Files :

        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Show
        Hudson added a comment - Integrated in Hadoop-Hdfs-0.23-Commit #645 (See https://builds.apache.org/job/Hadoop-Hdfs-0.23-Commit/645/ ) HDFS-1580 . Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864) Result = SUCCESS suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864 Files : /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Common-0.23-Commit #654 (See https://builds.apache.org/job/Hadoop-Common-0.23-Commit/654/)
        HDFS-1580. Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864)

        Result = SUCCESS
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864
        Files :

        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Show
        Hudson added a comment - Integrated in Hadoop-Common-0.23-Commit #654 (See https://builds.apache.org/job/Hadoop-Common-0.23-Commit/654/ ) HDFS-1580 . Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864) Result = SUCCESS suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864 Files : /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Mapreduce-0.23-Commit #655 (See https://builds.apache.org/job/Hadoop-Mapreduce-0.23-Commit/655/)
        HDFS-1580. Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864)

        Result = SUCCESS
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864
        Files :

        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Show
        Hudson added a comment - Integrated in Hadoop-Mapreduce-0.23-Commit #655 (See https://builds.apache.org/job/Hadoop-Mapreduce-0.23-Commit/655/ ) HDFS-1580 . Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864) Result = SUCCESS suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864 Files : /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Hdfs-trunk-Commit #1921 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk-Commit/1921/)
        Moving HDFS-2158, HDFS-2188, HDFS-2334, HDFS-2477, HDFS-2495, HDFS-2476, HDFS-1580, HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903)

        Result = SUCCESS
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Show
        Hudson added a comment - Integrated in Hadoop-Hdfs-trunk-Commit #1921 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk-Commit/1921/ ) Moving HDFS-2158 , HDFS-2188 , HDFS-2334 , HDFS-2477 , HDFS-2495 , HDFS-2476 , HDFS-1580 , HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903) Result = SUCCESS suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Common-trunk-Commit #1847 (See https://builds.apache.org/job/Hadoop-Common-trunk-Commit/1847/)
        Moving HDFS-2158, HDFS-2188, HDFS-2334, HDFS-2477, HDFS-2495, HDFS-2476, HDFS-1580, HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903)

        Result = SUCCESS
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Show
        Hudson added a comment - Integrated in Hadoop-Common-trunk-Commit #1847 (See https://builds.apache.org/job/Hadoop-Common-trunk-Commit/1847/ ) Moving HDFS-2158 , HDFS-2188 , HDFS-2334 , HDFS-2477 , HDFS-2495 , HDFS-2476 , HDFS-1580 , HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903) Result = SUCCESS suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Mapreduce-trunk-Commit #1855 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk-Commit/1855/)
        Moving HDFS-2158, HDFS-2188, HDFS-2334, HDFS-2477, HDFS-2495, HDFS-2476, HDFS-1580, HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903)

        Result = SUCCESS
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Show
        Hudson added a comment - Integrated in Hadoop-Mapreduce-trunk-Commit #1855 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk-Commit/1855/ ) Moving HDFS-2158 , HDFS-2188 , HDFS-2334 , HDFS-2477 , HDFS-2495 , HDFS-2476 , HDFS-1580 , HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903) Result = SUCCESS suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Hdfs-0.23-Build #190 (See https://builds.apache.org/job/Hadoop-Hdfs-0.23-Build/190/)
        HDFS-1580. Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864)

        Result = SUCCESS
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864
        Files :

        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Show
        Hudson added a comment - Integrated in Hadoop-Hdfs-0.23-Build #190 (See https://builds.apache.org/job/Hadoop-Hdfs-0.23-Build/190/ ) HDFS-1580 . Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864) Result = SUCCESS suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864 Files : /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Hdfs-trunk #977 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/977/)
        Moving HDFS-2158, HDFS-2188, HDFS-2334, HDFS-2477, HDFS-2495, HDFS-2476, HDFS-1580, HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903)

        Result = FAILURE
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Show
        Hudson added a comment - Integrated in Hadoop-Hdfs-trunk #977 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/977/ ) Moving HDFS-2158 , HDFS-2188 , HDFS-2334 , HDFS-2477 , HDFS-2495 , HDFS-2476 , HDFS-1580 , HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903) Result = FAILURE suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Mapreduce-0.23-Build #218 (See https://builds.apache.org/job/Hadoop-Mapreduce-0.23-Build/218/)
        HDFS-1580. Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864)

        Result = FAILURE
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864
        Files :

        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
        • /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Show
        Hudson added a comment - Integrated in Hadoop-Mapreduce-0.23-Build #218 (See https://builds.apache.org/job/Hadoop-Mapreduce-0.23-Build/218/ ) HDFS-1580 . Merging change r1210749, r1210602 from trunk to 0.23 (Revision 1297864) Result = FAILURE suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297864 Files : /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Mapreduce-trunk #1012 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/1012/)
        Moving HDFS-2158, HDFS-2188, HDFS-2334, HDFS-2477, HDFS-2495, HDFS-2476, HDFS-1580, HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903)

        Result = FAILURE
        suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903
        Files :

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        Show
        Hudson added a comment - Integrated in Hadoop-Mapreduce-trunk #1012 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/1012/ ) Moving HDFS-2158 , HDFS-2188 , HDFS-2334 , HDFS-2477 , HDFS-2495 , HDFS-2476 , HDFS-1580 , HDFS-1765 to 0.23.3 section in CHANGES.txt (Revision 1297903) Result = FAILURE suresh : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1297903 Files : /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

          People

          • Assignee:
            Jitendra Nath Pandey
            Reporter:
            Ivan Kelly
          • Votes:
            0 Vote for this issue
            Watchers:
            23 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development