[jira] [Commented] (HDFS-2803) Adding logging to LeaseRenewer for better lease expiration triage.
[ https://issues.apache.org/jira/browse/HDFS-2803?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187981#comment-13187981 ] Todd Lipcon commented on HDFS-2803: --- I think DEBUG level is more appropriate for both. Otherwise any time we run a dfs command like dfs -put, we'll end up with an INFO level message. Adding logging to LeaseRenewer for better lease expiration triage. -- Key: HDFS-2803 URL: https://issues.apache.org/jira/browse/HDFS-2803 Project: Hadoop HDFS Issue Type: Improvement Components: name-node Reporter: Jimmy Xiang Priority: Minor Labels: newbie It will be helpful to add some logging to LeaseRenewer when the daemon is terminated (Info level), and when the lease is renewed (Debug level). Since lacking logging, it is hard to know if a DFS client doesn't renew the lease because it hangs, or the lease renewer daemon is gone somehow. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2681) Add ZK client for leader election
[ https://issues.apache.org/jira/browse/HDFS-2681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187168#comment-13187168 ] Todd Lipcon commented on HDFS-2681: --- bq. So if your TCP disconnect timeouts are not set insanely high ( session timeout) then enterSafeMode will be called before session timeout expires and someone else becomes a master. This still isn't safe. For example, imagine the NN goes into a multi-minute GC pause just before writing an edit to its edit log. Since the GC pause is longer than the session timeout, some other NN will take over. Without active fencing, when the first NN wakes up, it will make that mutation to the edit log before it finds out about the ZK timeout. It sounds contrived but we've had many instances of data loss bugs in HBase due to scenarios like this in the past. Multi-minute GC pauses are rare but do happen. bq. It public because its a well defined property of the class. But it implies that external consumers of this class may want to directly manipulate the znode -- which is exposing an implementation detail unnecessarily. bq. Is the ALLCAPS on static strings a convention? You mean the member name should be all caps or the value? Yes, it's a convention that constants should have all-caps names. See the Sun java coding conventions, which we more-or-less follow: http://www.oracle.com/technetwork/java/codeconventions-135099.html#367 bq. So I need to have mock initialized before constructing the tester object. So I made mock a static member. But then java complained that inner classes cannot have static members. I'm not quite following - you already initialize the non-static {{mockZk}} in {{TestActiveStandbyElector.init()}}?. Then if it's a non-static inner class, it can simply refer to the already-initialized member of its outer class. bq. Could you please point me to some place which explains what to log at different log levels? I don't think we have any formal guidelines here.. the basic assumptions I make are: - ERROR: unrecoverable errors (eg some block is apparently lost, or a failover failed, etc) - WARN: recoverable errors (eg failures that will be retried, blocks that have become under-replicated but can be repaired, etc) - INFO: normal operations proceeding as expected, but interesting enough that operators will want to see it. - DEBUG: information that will be useful to developers debugging unit tests or running small test clusters (unit tests generally enable these, but users generally don't). Also handy when you have a reproducible bug on the client - you can ask the user to enable DEBUG and re-run, for example. - TRACE: super-detailed trace information that will only be enabled in rare circumstances. We don't use this much. Add ZK client for leader election - Key: HDFS-2681 URL: https://issues.apache.org/jira/browse/HDFS-2681 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Suresh Srinivas Assignee: Bikas Saha Fix For: HA branch (HDFS-1623) Attachments: HDFS-2681.HDFS-1623.patch, HDFS-2681.HDFS-1623.patch, HDFS-2681.HDFS-1623.patch, HDFS-2681.HDFS-1623.patch, Zookeeper based Leader Election and Monitoring Library.pdf ZKClient needs to support the following capabilities: # Ability to create a znode for co-ordinating leader election. # Ability to monitor and receive call backs when active znode status changes. # Ability to get information about the active node. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2798) Append may race with datanode block scanner, causing replica to be incorrectly marked corrupt
[ https://issues.apache.org/jira/browse/HDFS-2798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187170#comment-13187170 ] Todd Lipcon commented on HDFS-2798: --- You can reproduce this fairly reliably by adding a sleep call in BlockReceiver as follows: {code} case PIPELINE_SETUP_APPEND: replicaInfo = datanode.data.append(block, newGs, minBytesRcvd); try { Thread.sleep(1000); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } if (datanode.blockScanner != null) { // remove from block scanner datanode.blockScanner.deleteBlock(block.getBlockPoolId(), block.getLocalBlock()); } block.setGenerationStamp(newGs); break; {code} TestFileAppend2 will then time out. Append may race with datanode block scanner, causing replica to be incorrectly marked corrupt - Key: HDFS-2798 URL: https://issues.apache.org/jira/browse/HDFS-2798 Project: Hadoop HDFS Issue Type: Bug Components: data-node Affects Versions: 0.22.0, 0.23.0 Reporter: Todd Lipcon Priority: Critical When a pipeline is setup for append, the block's metadata file is renamed before the block is removed from the datanode block scanner queues. This can cause a race condition where the block scanner incorrectly marks the block as corrupt, since it tries to scan the file corresponding to the old genstamp. This causes TestFileAppend2 to time out in extremely rare circumstances - the corrupt replica prevents the writer thread from completing the file. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2742) HA: observed dataloss in replication stress test
[ https://issues.apache.org/jira/browse/HDFS-2742?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187207#comment-13187207 ] Todd Lipcon commented on HDFS-2742: --- I also ran the replication stress test for 10x as long as normal and had no block loss. I'll loop this for a while to make sure it's really stable. HA: observed dataloss in replication stress test Key: HDFS-2742 URL: https://issues.apache.org/jira/browse/HDFS-2742 Project: Hadoop HDFS Issue Type: Sub-task Components: data-node, ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Blocker Attachments: hdfs-2742.txt, hdfs-2742.txt, hdfs-2742.txt, log-colorized.txt The replication stress test case failed over the weekend since one of the replicas went missing. Still diagnosing the issue, but it seems like the chain of events was something like: - a block report was generated on one of the nodes while the block was being written - thus the block report listed the block as RBW - when the standby replayed this queued message, it was replayed after the file was marked complete. Thus it marked this replica as corrupt - it asked the DN holding the corrupt replica to delete it. And, I think, removed it from the block map at this time. - That DN then did another block report before receiving the deletion. This caused it to be re-added to the block map, since it was FINALIZED now. - Replication was lowered on the file, and it counted the above replica as non-corrupt, and asked for the other replicas to be deleted. - All replicas were lost. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2747) HA: entering safe mode after starting SBN can NPE
[ https://issues.apache.org/jira/browse/HDFS-2747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187213#comment-13187213 ] Todd Lipcon commented on HDFS-2747: --- +1, looks good to me. I'll double-check the tests pass and commit momentarily. HA: entering safe mode after starting SBN can NPE - Key: HDFS-2747 URL: https://issues.apache.org/jira/browse/HDFS-2747 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Uma Maheswara Rao G Attachments: HDFS-2747.patch, HDFS-2747.patch Entering Safemode on the primary after while it's already in safemode after the SBN is started results in an NPE: {noformat} hadoop-0.24.0-SNAPSHOT $ ./bin/hdfs dfsadmin -safemode get Safe mode is ON hadoop-0.24.0-SNAPSHOT $ ./bin/hdfs dfsadmin -safemode enter safemode: java.lang.NullPointerException {noformat} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2772) HA: On transition to active, standby should not swallow ELIE
[ https://issues.apache.org/jira/browse/HDFS-2772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187246#comment-13187246 ] Todd Lipcon commented on HDFS-2772: --- +1, lgtm. HA: On transition to active, standby should not swallow ELIE Key: HDFS-2772 URL: https://issues.apache.org/jira/browse/HDFS-2772 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Aaron T. Myers Attachments: HDFS-2772-HDFS-1623.patch EditLogTailer#doTailEdits currently catches, logs, and swallows EditLogInputException. This is fine in the case when the standby is sitting idly behind tailing logs. However, when the standby is transitioning to active, swallowing this exception is incorrect, since it could cause the standby to silently fail to load all the edits before becoming active. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2691) HA: Tests and fixes for pipeline targets and replica recovery
[ https://issues.apache.org/jira/browse/HDFS-2691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187256#comment-13187256 ] Todd Lipcon commented on HDFS-2691: --- In order to fix this, we need to get the {{targets}} list for incomplete blocks replicated to the SBN. The most straightforward way to do this involves the DNs somehow reporting the RBW replicas to both the SBN and the ANN. When the NN receives a reported RBW replica, it adds to the targets list for an incomplete block, thus allowing a recovery to succeed even after fail-over. Two potential ways to get the RBW replicas reported: *Solution 1*: whenever any pipeline is opened on the DNs, it reports the RBW replica to the NNs -- similar to how it reports FINALIZED replicas on close. This would be queued to both the active and standby namenodes. Advantage: timely reporting of replicas. Disadvantage: adds slightly to NN load even in common case. *Solution 2*: when the DN detects that a failover has occurred to a new Active, it reports all RBW replicas to the new active at that point. Advantage: doesn't add to non-failover load. Disadvantage: there is some complexity in that there is a window where the new active is in active state but hasn't yet heard from all the DNs. Thus, it will serve incorrect reads or incorrectly handle a lease recovery for a short window of time (since it doesn't know any locations for the last block). Given the potential for user-visible incorrect results for a window of time in solution 2, I plan to pursue solution 1, unless someone has a better idea. HA: Tests and fixes for pipeline targets and replica recovery - Key: HDFS-2691 URL: https://issues.apache.org/jira/browse/HDFS-2691 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Critical Attachments: hdfs-2691.txt Currently there are some TODOs around pipeline/recovery code in the HA branch. For example, commitBlockSynchronization only gets sent to the active NN which may have failed over by that point. So, we need to write some tests here and figure out what the correct behavior is. Another related area is the treatment of targets in the pipeline. When a pipeline is created, the active NN adds the expected locations to the BlockInfoUnderConstruction, but the DN identifiers aren't logged with the OP_ADD. So after a failover, the BlockInfoUnderConstruction will have no targets and I imagine replica recovery would probably trigger some issues. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2767) HA: ConfiguredFailoverProxyProvider should support NameNodeProtocol
[ https://issues.apache.org/jira/browse/HDFS-2767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187302#comment-13187302 ] Todd Lipcon commented on HDFS-2767: --- {code} +} catch (RuntimeException rte) { + throw rte; {code} doesn't serve any purpose - The functions createFailoverProxyWithNamenodeProtocol and createFailoverProxyWithClientProtocol are identical except for the class they take, right? Why not just make it take the interface class as a third parameter? - Can you update NameNodeConnector.createNamenode to call createNNProxyWithNamenodeProtocol in this patch, just so we don't introduce new dup code here? Thanks! HA: ConfiguredFailoverProxyProvider should support NameNodeProtocol --- Key: HDFS-2767 URL: https://issues.apache.org/jira/browse/HDFS-2767 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, hdfs client Affects Versions: HA branch (HDFS-1623) Reporter: Uma Maheswara Rao G Assignee: Uma Maheswara Rao G Priority: Blocker Attachments: HDFS-2767.patch, HDFS-2767.patch, hdfs-2767-what-todd-had.txt Presentely ConfiguredFailoverProxyProvider supports ClinetProtocol. It should support NameNodeProtocol also, because Balancer uses NameNodeProtocol for getting blocks. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2767) HA: ConfiguredFailoverProxyProvider should support NameNodeProtocol
[ https://issues.apache.org/jira/browse/HDFS-2767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187340#comment-13187340 ] Todd Lipcon commented on HDFS-2767: --- I meant that, in this patch, you can make NameNodeConnector use the new API {{createNNProxyWithNamenodeProtocol}} - as it's just refactoring. Then, in the other patch, you can integrate it with the failover proxy creation. HA: ConfiguredFailoverProxyProvider should support NameNodeProtocol --- Key: HDFS-2767 URL: https://issues.apache.org/jira/browse/HDFS-2767 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, hdfs client Affects Versions: HA branch (HDFS-1623) Reporter: Uma Maheswara Rao G Assignee: Uma Maheswara Rao G Priority: Blocker Attachments: HDFS-2767.patch, HDFS-2767.patch, hdfs-2767-what-todd-had.txt Presentely ConfiguredFailoverProxyProvider supports ClinetProtocol. It should support NameNodeProtocol also, because Balancer uses NameNodeProtocol for getting blocks. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2592) HA: Balancer support for HA namenodes
[ https://issues.apache.org/jira/browse/HDFS-2592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187411#comment-13187411 ] Todd Lipcon commented on HDFS-2592: --- Looks mostly good. One small nit - can you add some javadoc to HATestUtil.setFailoverConfigurations? Also, need to update the patch to apply on current branch. Thanks Uma! HA: Balancer support for HA namenodes - Key: HDFS-2592 URL: https://issues.apache.org/jira/browse/HDFS-2592 Project: Hadoop HDFS Issue Type: Sub-task Components: balancer, ha Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Uma Maheswara Rao G Attachments: HDFS-2592.patch, HDFS-2592.patch, HDFS-2592.patch The balancer currently interacts directly with namenode InetSocketAddresses and makes its own IPC proxies. We need to integrate it with HA so that it uses the same client failover infrastructure. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2795) HA: Standby NN takes a long time to recover from a dead DN starting up
[ https://issues.apache.org/jira/browse/HDFS-2795?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187467#comment-13187467 ] Todd Lipcon commented on HDFS-2795: --- Woops, this broke one of the TestPersistBlocks tests -- when it replays append, it was getting a NPE since {{haContext}} isn't set yet during startup. Does the following look OK? {code} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/ap index 5e8377e..c57d152 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -3718,7 +3718,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats, @Override public boolean isPopulatingReplQueues() { -if (!haContext.getState().shouldPopulateReplQueues()) { +if (haContext != null // null during startup! +!haContext.getState().shouldPopulateReplQueues()) { return false; } // safeMode is volatile, and may be set to null at any time {code} HA: Standby NN takes a long time to recover from a dead DN starting up -- Key: HDFS-2795 URL: https://issues.apache.org/jira/browse/HDFS-2795 Project: Hadoop HDFS Issue Type: Sub-task Components: data-node, ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Todd Lipcon Priority: Critical Fix For: HA branch (HDFS-1623) Attachments: hdfs-2795.txt To reproduce: # Start an HA cluster with a DN. # Write several blocks to the FS with replication 1. # Shutdown the DN # Wait for the NNs to declare the DN dead. All blocks will be under-replicated. # Restart the DN. Note that upon restarting the DN, the active NN will immediately get all block locations from the initial BR. The standby NN will not, and instead will slowly add block locations for a subset of the previously-missing blocks on every DN heartbeat. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2747) HA: entering safe mode after starting SBN can NPE
[ https://issues.apache.org/jira/browse/HDFS-2747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13186659#comment-13186659 ] Todd Lipcon commented on HDFS-2747: --- - add braces around the body of the new if statement - typo in test name: ShoulNot - parameter should capitalize namesystem not nameSystem since it is FSNamesystem, not FSNameSystem -- I also think NameNodeAdapter is probably a better spot for this than FSImageTestUtil - In the javadoc for the new tests, can you mention that it is a regression test for HDFS-2747? I think it's useful for future maintenance of the tests to be able to refer back to the discussion on this bug. Thanks! HA: entering safe mode after starting SBN can NPE - Key: HDFS-2747 URL: https://issues.apache.org/jira/browse/HDFS-2747 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Uma Maheswara Rao G Attachments: HDFS-2747.patch Entering Safemode on the primary after while it's already in safemode after the SBN is started results in an NPE: {noformat} hadoop-0.24.0-SNAPSHOT $ ./bin/hdfs dfsadmin -safemode get Safe mode is ON hadoop-0.24.0-SNAPSHOT $ ./bin/hdfs dfsadmin -safemode enter safemode: java.lang.NullPointerException {noformat} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2681) Add ZK client for leader election
[ https://issues.apache.org/jira/browse/HDFS-2681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13186672#comment-13186672 ] Todd Lipcon commented on HDFS-2681: --- - Can {{ActiveStandbyElector}} be made package-private? If not, it should get audience annotations (perhaps private, perhaps LimitedPrivate to HDFS? not sure) -- Same with the inner interface {code} + * or loss of Zookeeper quorum. Thus enterSafeMode can be used to guard + * against split-brain issues. In such situations it might be prudent to + * call becomeStandby too. However, such state change operations might be + * expensive and enterSafeMode can help guard against doing that for + * transient issues. + */ {code} - I think the above references to {{enterSafeMode}} are supposed to be {{enterNeutralMode}}, right? - Also, it can't really guard against split brain, because there is no guarantee on the timeliness of delivery of these messages. That is to say, the other participants in the election might receive {{becomeActive}} before this participant receives {{enterNeutralMode}}. So, I'm not sold on the necessity of this callback. {code} +void notifyFatalError(); {code} Shouldn't this come with some kind of Exception argument or at least a String error message? Right now if we hit it, it won't be clear in the logs which of several cases caused it. {code} + /** + * Name of the lock znode used by the library + */ + public static final String lockFileName = ActiveStandbyElectorLock-21EC2020-3AEA-1069-A2DD-08002B30309D; {code} - why is this public? - should also be ALL_CAPS. - what's with the random UUID in there? Assumedly this library would be configured to be rooted inside some base directory in the tree which would include the namespaceID, etc. {code} + * Setting a very short session timeout may result in frequent transitions + * between active and standby states during issues like network outages. {code} Also should mention GC pauses here -- they're more frequent than network blips IME. {code} + * @param zookeeperHostPort + * ZooKeeper hostPort for all ZooKeeper servers {code} Comma-separated? Perhaps better to name it {{zookeeperHostPorts}} since there is more than one server in the quorum. - typo: reference to callback *inteface* object {code} +appData = new byte[data.length]; +System.arraycopy(data, 0, appData, 0, data.length); {code} Could use Arrays.copyOf() here instead - Rename {{operationSuccess}} etc to {{isSuccessCode}} -- I think that's a clearer naming. - Make ActiveStandbyElectorTester an inner class of TestActiveStandbyElector. We generally discourage having multiple outer classes per Java file. You can then avoid making two mockZk objects, and count wouldn't have to be static, either. The whole class could be done as an anonymous class, inline, probably. - Echo what Suresh said about catching exceptions in tests - should let it fall through and fail the test - that'll also make sure the exception that was triggered makes it all the way up to the test runner and recorded properly (handy when debugging in Eclipse for example) - In a couple places, you catch an expected exception and then verify, but you should also add an {{Assert.fail(Didn't throw exception)}} in the {{try}} clause to make sure the exception was actually thrown. {code} + * active and the rest become standbys. /br This election mechanism is + * efficient for small number of election candidates (order of 10's) because {code} Should say _only_ efficient to be clear {code} + * {@link ActiveStandbyElectorCallback} to interact with the elector + * + */ {code} Extra blank lines inside javadoc comments should be removed Some general notes/nits: - Some of the INFO level logs are probably better off at DEBUG level. Or else, they should be expanded out to more operator-readable information (most ops will have no clue what CreateNode result: 2 for path: /blah/blah means. - Some more DEBUG level logs could be added to the different cases, or even INFO level ones at the unexpected ones (like having to retry, or being Disconnected, etc). I don't think there's any harm in being fairly verbose about state change events that are expected to only happen during fail-overs, and in case it goes wrong we want to have all the details at hand. But, as above, they should be operator-understandable. - Javadoc breaks should be {{br/}} rather than {{/br}}. - Constants should be ALL_CAPS -- eg {{LOG}} rather than {{Log} - Add a constant for NUM_RETRIES instead of hard-coded 3. - Should never use {{e.printStackTrace}} -- instead, use {{LOG.error}} or {{LOG.warn}} with the second argument as the exception. This will print the trace, but also makes sure it goes to the right log. Add ZK client for leader election
[jira] [Commented] (HDFS-2794) HA: Active NN may purge edit log files before standby NN has a chance to read them
[ https://issues.apache.org/jira/browse/HDFS-2794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13186739#comment-13186739 ] Todd Lipcon commented on HDFS-2794: --- Worth noting that this only happens if the admin explicitly invokes saveNamespace on the active node more times than the configured retention count (or restarts serveral times without running the SBN in between). So it's easy to work-around by configuring the retention count high, and in the scenario that you do hit the problem, you can simply scp any image from the active and restart the SBN. HA: Active NN may purge edit log files before standby NN has a chance to read them -- Key: HDFS-2794 URL: https://issues.apache.org/jira/browse/HDFS-2794 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Aaron T. Myers Given that the active NN is solely responsible for purging finalized edit log segments, and given that the active NN has no way of knowing when the standby reads edit logs, it's possible that the standby NN could fail to read all edits it needs before the active purges the files. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2791) If block report races with closing of file, replica is incorrectly marked corrupt
[ https://issues.apache.org/jira/browse/HDFS-2791?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13186407#comment-13186407 ] Todd Lipcon commented on HDFS-2791: --- To start brainstorming a solution, here are a few scattered thoughts: - The basic requirement is that we should always accept a past valid state of the block, since we don't have any global barrier before the block is marked complete. So we need to support RBW-before-finalized even if it has a too-short length, for example. There might be other bugs similar to this if the file is re-opened for append (eg a block is reported with a too-young generation stamp racing with the re-open). - I think this can only happen in the _first_ block report after a block's state changes, since each block report freshly examines the DN state. So maybe we can use the block report timestamps to our advantage somehow? ie if the block changed state in between the previous block report was received and this one, the BR might have raced? The floor is open for creative simple solutions :) If block report races with closing of file, replica is incorrectly marked corrupt - Key: HDFS-2791 URL: https://issues.apache.org/jira/browse/HDFS-2791 Project: Hadoop HDFS Issue Type: Bug Components: data-node, name-node Affects Versions: 0.22.0, 0.23.0 Reporter: Todd Lipcon The following sequence of events results in a replica mistakenly marked corrupt: 1. Pipeline is open with 2 replicas 2. DN1 generates a block report but is slow in sending to the NN (eg some flaky network). It gets stuck right before the block report RPC. 3. Client closes the file. 4. DN2 is fast and sends blockReceived to the NN. NN marks the block as COMPLETE 5. DN1's block report proceeds, and includes the block in an RBW state. 6. (x) NN incorrectly marks the replica as corrupt, since it is an RBW replica on a COMPLETE block. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2731) Autopopulate standby name dirs if they're empty
[ https://issues.apache.org/jira/browse/HDFS-2731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13185751#comment-13185751 ] Todd Lipcon commented on HDFS-2731: --- bq. I am missing this: both Image and Edits should be on shared dirs (ie no need to copy image from primary - it should be available in shared dir). Since we generalized the edit log interface, we can't store images on the shared edit storage. We only get the stream abstraction for it. We could also extend the system to require a shared *image* dir, but then we couldn't support BK-based HA. (In the future when we can store checkpoints in HDFS itself we wouldn't have that issue) Autopopulate standby name dirs if they're empty --- Key: HDFS-2731 URL: https://issues.apache.org/jira/browse/HDFS-2731 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Eli Collins To setup a SBN we currently format the primary then manually copy the name dirs to the SBN. The SBN should do this automatically. Specifically, on NN startup, if HA with a shared edits dir is configured and populated, if the SBN has empty name dirs it should downloads the image and log from the primary (as an optimization it could copy the logs from the shared dir). If the other NN is still in standby then it should fail to start as it does currently. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2747) HA: entering safe mode after starting SBN can NPE
[ https://issues.apache.org/jira/browse/HDFS-2747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13185076#comment-13185076 ] Todd Lipcon commented on HDFS-2747: --- Ah, that makes sense. Good sleuthing. Planning to provide a patch? HA: entering safe mode after starting SBN can NPE - Key: HDFS-2747 URL: https://issues.apache.org/jira/browse/HDFS-2747 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Uma Maheswara Rao G Entering Safemode on the primary after while it's already in safemode after the SBN is started results in an NPE: {noformat} hadoop-0.24.0-SNAPSHOT $ ./bin/hdfs dfsadmin -safemode get Safe mode is ON hadoop-0.24.0-SNAPSHOT $ ./bin/hdfs dfsadmin -safemode enter safemode: java.lang.NullPointerException {noformat} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2592) HA: Balancer support for HA namenodes
[ https://issues.apache.org/jira/browse/HDFS-2592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183486#comment-13183486 ] Todd Lipcon commented on HDFS-2592: --- Uma, do you mind if I take this over to finish up your patch? I was planning on working on HDFS-2767 which is closely related. HA: Balancer support for HA namenodes - Key: HDFS-2592 URL: https://issues.apache.org/jira/browse/HDFS-2592 Project: Hadoop HDFS Issue Type: Sub-task Components: balancer, ha Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Uma Maheswara Rao G Attachments: HDFS-2592.patch The balancer currently interacts directly with namenode InetSocketAddresses and makes its own IPC proxies. We need to integrate it with HA so that it uses the same client failover infrastructure. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2775) HA: TestStandbyCheckpoints.testBothNodesInStandbyState fails intermittently
[ https://issues.apache.org/jira/browse/HDFS-2775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183502#comment-13183502 ] Todd Lipcon commented on HDFS-2775: --- Yes, this just a test race. The issue is that the checkpoint is saved to storage, and only after that is {{mostRecentCheckpointTxId}} updated. So, the test sees the checkpoint and then the assert fails. We should probably fix this with some simple synchronization - but it's only a test problem and not a code issue. HA: TestStandbyCheckpoints.testBothNodesInStandbyState fails intermittently --- Key: HDFS-2775 URL: https://issues.apache.org/jira/browse/HDFS-2775 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, test Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon This test is failing periodically on this assertion: {code} assertEquals(12, nn0.getNamesystem().getFSImage().getStorage() .getMostRecentCheckpointTxId()); {code} My guess is it's a test race. Investigating... -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2753) Standby namenode stuck in safemode during a failover
[ https://issues.apache.org/jira/browse/HDFS-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183626#comment-13183626 ] Todd Lipcon commented on HDFS-2753: --- The test adds blocks while the SBN is down. This makes them get queued up in the block received list of that BPServiceActor. When it restarts, the DN calls register(), followed by reportReceivedDeletedBlocks(), followed by blockReport(). So the received blocks always show up first. If you comment out the fix, the test case reliably fails with the error you described (stuck in safemode). Standby namenode stuck in safemode during a failover Key: HDFS-2753 URL: https://issues.apache.org/jira/browse/HDFS-2753 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Hari Mankude Assignee: Hari Mankude Attachments: HDFS-2753.patch, hdfs-2753.txt Write traffic initiated from the client. Manual failover is done by killing NN and converting a different standby to active. NN is restarted as standby. The restarted standby stays in safemode forever. More information in the description. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2767) HA: ConfiguredFailoverProxyProvider should support NameNodeProtocol
[ https://issues.apache.org/jira/browse/HDFS-2767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183867#comment-13183867 ] Todd Lipcon commented on HDFS-2767: --- Hi Uma. I had started working on this before you posted your patch, but looks like we went a similar direction. The only suggestion I have is to make the interface an argument of the constructor rather than calling a setter after it's instantiated. I'll upload what I have - do you think you could make that change in your patch? Also, regarding this section: {code} +// TODO(HA): Need other way to create the proxy instance based on +// protocol here. +if (protocol != null NamenodeProtocol.class.equals(protocol)) { + current.namenode = DFSUtil.createNamenodeWithNNProtocol( + current.address, conf); +} else { {code} I think you can remove the TODO and change the {{else}} to an {{else if}} to check for ClientProtocol, with a final {{else}} clause that throws an AssertionError or IllegalStateException. Lastly, I think we do need to wire the {{ugi}} parameter in to {{createNamenodeWithNNProtocol}} or else after a failover the user accessing HDFS might accidentally switch! HA: ConfiguredFailoverProxyProvider should support NameNodeProtocol --- Key: HDFS-2767 URL: https://issues.apache.org/jira/browse/HDFS-2767 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, hdfs client Affects Versions: HA branch (HDFS-1623) Reporter: Uma Maheswara Rao G Assignee: Todd Lipcon Priority: Blocker Attachments: HDFS-2767.patch, hdfs-2767-what-todd-had.txt Presentely ConfiguredFailoverProxyProvider supports ClinetProtocol. It should support NameNodeProtocol also, because Balancer uses NameNodeProtocol for getting blocks. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2592) HA: Balancer support for HA namenodes
[ https://issues.apache.org/jira/browse/HDFS-2592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183876#comment-13183876 ] Todd Lipcon commented on HDFS-2592: --- This looks fairly reasonable. A few items: - Is it possible to move that new code out of the NameNodeConnector constructor into a static method in DFSUtil or even DFSClient? - Rather than duplicating the code to parse the maxFailoverAttempts, failoverBaseSleepMillis, etc, can we reuse some of the code that's in DFSClient? If we move the connection code into a static method in DFSClient, then we can instantiate a DFSClient.Conf and pull out the variables from there, for example. - Some too-long lines in the new test code - The new test is mostly dup code from TestBalancer. Is it possible to reuse more of the code by refactoring into static methods, etc? - Similarly much of the setup code is duplicated from HAUtil.configureFailoverFs. Can you just call that function, then grab the conf from the resulting filesystem, or refactor that method so you can reuse the configuration generating code? HA: Balancer support for HA namenodes - Key: HDFS-2592 URL: https://issues.apache.org/jira/browse/HDFS-2592 Project: Hadoop HDFS Issue Type: Sub-task Components: balancer, ha Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Uma Maheswara Rao G Attachments: HDFS-2592.patch, HDFS-2592.patch The balancer currently interacts directly with namenode InetSocketAddresses and makes its own IPC proxies. We need to integrate it with HA so that it uses the same client failover infrastructure. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2773) HA: reading edit logs from an earlier version leaves blocks in under-construction state
[ https://issues.apache.org/jira/browse/HDFS-2773?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183894#comment-13183894 ] Todd Lipcon commented on HDFS-2773: --- I added the following: {code} + // OP_CLOSE should add finalized blocks. This code path + // is only executed when loading edits written by prior + // versions of Hadoop. Current versions always log + // OP_ADD operations as each block is allocated. + newBI = new BlockInfo(newBlock, file.getReplication()); {code} Will commit momentarily. HA: reading edit logs from an earlier version leaves blocks in under-construction state --- Key: HDFS-2773 URL: https://issues.apache.org/jira/browse/HDFS-2773 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Blocker Attachments: hadoop-1.0-multiblock-file.tgz, hdfs-2773.txt In HDFS-2602, the code for applying OP_ADD and OP_CLOSE was changed a bit, and the new code has the following problem: if an OP_CLOSE includes new blocks (ie not previously seen in an OP_ADD) then those blocks will remain in the under construction state rather than being marked complete. This is because {{updateBlocks}} always creates {{BlockInfoUnderConstruction}} regardless of the opcode. This bug only affects the upgrade path, since in trunk we always persist blocks with OP_ADDs before we call OP_CLOSE. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2738) FSEditLog.selectinputStreams is reading through in-progress streams even when non-in-progress are requested
[ https://issues.apache.org/jira/browse/HDFS-2738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183899#comment-13183899 ] Todd Lipcon commented on HDFS-2738: --- +1, looks good to me. Thanks for making those changes. FSEditLog.selectinputStreams is reading through in-progress streams even when non-in-progress are requested --- Key: HDFS-2738 URL: https://issues.apache.org/jira/browse/HDFS-2738 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Blocker Attachments: HDFS-2738-HDFS-1623.patch, HDFS-2738-HDFS-1623.patch, HDFS-2738-HDFS-1623.patch The new code in HDFS-1580 is causing an issue with selectInputStreams in the HA context. When the active is writing to the shared edits, selectInputStreams is called on the standby. This ends up calling {{journalSet.getInputStream}} but doesn't pass the {{inProgressOk=false}} flag. So, {{getInputStream}} ends up reading and validating the in-progress stream unnecessarily. Since the validation results are no longer properly cached, {{findMaxTransaction}} also re-validates the in-progress stream, and then breaks the corruption check in this code. The end result is a lot of errors like: 2011-12-30 16:45:02,521 ERROR namenode.FileJournalManager (FileJournalManager.java:getNumberOfTransactions(266)) - Gap in transactions, max txnid is 579, 0 txns from 578 2011-12-30 16:45:02,521 INFO ha.EditLogTailer (EditLogTailer.java:run(163)) - Got error, will try again. java.io.IOException: No non-corrupt logs for txid 578 at org.apache.hadoop.hdfs.server.namenode.JournalSet.getInputStream(JournalSet.java:229) at org.apache.hadoop.hdfs.server.namenode.FSEditLog.selectInputStreams(FSEditLog.java:1081) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.doTailEdits(EditLogTailer.java:115) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.access$0(EditLogTailer.java:100) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread.run(EditLogTailer.java:154) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2775) HA: TestStandbyCheckpoints.testBothNodesInStandbyState fails intermittently
[ https://issues.apache.org/jira/browse/HDFS-2775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183900#comment-13183900 ] Todd Lipcon commented on HDFS-2775: --- bq. Should FSImage#getMostRecentCheckpointTxId perhaps be marked @VisibleForTesting? Eh, I don't see any reason it shouldn't be used elsewhere in the code either. I generally try to only use that when you're exposing some piece of internal state that shouldn't normally be used from the main non-test code. HA: TestStandbyCheckpoints.testBothNodesInStandbyState fails intermittently --- Key: HDFS-2775 URL: https://issues.apache.org/jira/browse/HDFS-2775 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, test Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Fix For: HA branch (HDFS-1623) Attachments: hdfs-2775.txt This test is failing periodically on this assertion: {code} assertEquals(12, nn0.getNamesystem().getFSImage().getStorage() .getMostRecentCheckpointTxId()); {code} My guess is it's a test race. Investigating... -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2766) HA: test for case where standby partially reads log and then performs checkpoint
[ https://issues.apache.org/jira/browse/HDFS-2766?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183902#comment-13183902 ] Todd Lipcon commented on HDFS-2766: --- +1 lgtm. HA: test for case where standby partially reads log and then performs checkpoint Key: HDFS-2766 URL: https://issues.apache.org/jira/browse/HDFS-2766 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Attachments: HDFS-2766-HDFS-1623.patch, HDFS-2766-HDFS-1623.patch Here's a potential bug case that we don't currently test for: - SBN is reading a finalized edits file when NFS disappears halfway through (or some intermittent error happens) - SBN performs a checkpoint and uploads it to the NN - NN receives a checkpoint that doesn't correspond to the end of any log segment - Both NN and SBN should be able to restart at this point. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2742) HA: observed dataloss in replication stress test
[ https://issues.apache.org/jira/browse/HDFS-2742?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183911#comment-13183911 ] Todd Lipcon commented on HDFS-2742: --- bq. What is the implication of ignoring RBW altogether at the standby? That's an idea I've thought a little about, but I think it has some implications for lease recovery. In actuality, in order to fix the cases in HDFS-2691, I think we need to send RBW blockReceived messages to the SBN as soon as a pipeline is constructed. I do like it, though, as at least a stop-gap for now while we work on a more thorough solution. bq. If editlog has a finalized record, can we just ignore the RBW from the block report? Possibly - I haven't thought through the whole Append state machine. I assumed that the code that marks a RBW replica as corrupt when received for a COMPLETED block is probably there for a good reason... so changing the behavior there might introduce some other bugs that could even hurt the non-HA case. I'm going to keep working on this and see if I can come up with a simpler solution based on some of Suresh's ideas above. HA: observed dataloss in replication stress test Key: HDFS-2742 URL: https://issues.apache.org/jira/browse/HDFS-2742 Project: Hadoop HDFS Issue Type: Sub-task Components: data-node, ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Blocker Attachments: hdfs-2742.txt, log-colorized.txt The replication stress test case failed over the weekend since one of the replicas went missing. Still diagnosing the issue, but it seems like the chain of events was something like: - a block report was generated on one of the nodes while the block was being written - thus the block report listed the block as RBW - when the standby replayed this queued message, it was replayed after the file was marked complete. Thus it marked this replica as corrupt - it asked the DN holding the corrupt replica to delete it. And, I think, removed it from the block map at this time. - That DN then did another block report before receiving the deletion. This caused it to be re-added to the block map, since it was FINALIZED now. - Replication was lowered on the file, and it counted the above replica as non-corrupt, and asked for the other replicas to be deleted. - All replicas were lost. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2753) Standby namenode stuck in safemode during a failover
[ https://issues.apache.org/jira/browse/HDFS-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13182746#comment-13182746 ] Todd Lipcon commented on HDFS-2753: --- It seems like this should be easy to produce -- just so long as there is some write traffic while one of the nodes is down, then the blockReceived calls will get queued up in the BPServiceActor, and when it re-registers they will all be flushed before the first block report, triggering the bug. I will try to write a test. Standby namenode stuck in safemode during a failover Key: HDFS-2753 URL: https://issues.apache.org/jira/browse/HDFS-2753 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Hari Mankude Assignee: Hari Mankude Attachments: HDFS-2753.patch Write traffic initiated from the client. Manual failover is done by killing NN and converting a different standby to active. NN is restarted as standby. The restarted standby stays in safemode forever. More information in the description. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2742) HA: observed dataloss in replication stress test
[ https://issues.apache.org/jira/browse/HDFS-2742?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13182787#comment-13182787 ] Todd Lipcon commented on HDFS-2742: --- This seems to have caused some issues in TestHASafeMode. I'll upload a new rev later today. HA: observed dataloss in replication stress test Key: HDFS-2742 URL: https://issues.apache.org/jira/browse/HDFS-2742 Project: Hadoop HDFS Issue Type: Sub-task Components: data-node, ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Blocker Attachments: hdfs-2742.txt, log-colorized.txt The replication stress test case failed over the weekend since one of the replicas went missing. Still diagnosing the issue, but it seems like the chain of events was something like: - a block report was generated on one of the nodes while the block was being written - thus the block report listed the block as RBW - when the standby replayed this queued message, it was replayed after the file was marked complete. Thus it marked this replica as corrupt - it asked the DN holding the corrupt replica to delete it. And, I think, removed it from the block map at this time. - That DN then did another block report before receiving the deletion. This caused it to be re-added to the block map, since it was FINALIZED now. - Replication was lowered on the file, and it counted the above replica as non-corrupt, and asked for the other replicas to be deleted. - All replicas were lost. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2742) HA: observed dataloss in replication stress test
[ https://issues.apache.org/jira/browse/HDFS-2742?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13182907#comment-13182907 ] Todd Lipcon commented on HDFS-2742: --- Looking into this has made me aware of a lurking can of worms... the summary of the issue is that we have to make sure the interleaving of block state transitions and messages from the datanode is maintained. The sequence needs to look like: 1. Block is allocated 2. Block reports may arrive with the block in RBW state 3. Block is completed 4. Block reports and blockReceived messages may arrive with the block in FINALIZED state. On the active node, this sequence is guaranteed since we only mark a block as complete once the minimum number of replicas has reported FINALIZED. And once any replica reports FINALIZED, we shouldn't see anymore RBW replicas, unless they're truly corrupt. On the standby node, though, the application of the edits are delayed until it reads the shared storage log. So it may receive step 2 and step 4 long before it even knows about the block. The trick is that we need to interleave them into the correct position in the edits stream. The issue in this JIRA is that, in the tip of the branch today, we are processing all queued messages after applying all the edits. So, if we received a block report with an RBW replica, it will be processed after the replica is already completed, thus swapping step 2 and 3 in the above sequence. This results in the block being marked as corrupt. If instead we try to process the queued messages as soon as we first hear about the block, we have the opposite problem -- step 3 and step 4 are switched. This causes problems for Safe Mode since it isn't properly accounting the number of complete blocks in that case. Hence the patch currently attached to this JIRA breaks TestHASafeMode. I spent the afternoon thinking about it, and the best solution I can come up with is the following: - rather than a single PendingDatanodeMessages queue, where we queue up the entire block report or blockReceived message, we should make the queueing more fine-grained. So, if we receive a block report, we can open it up and handle each block separately. For each block, we have a few cases: -- *correct state*: the replica has the right genstamp and a consistent state - eg an RBW replica for an in-progress block or a FINALIZED replica for a completed block. We can handle these immediately. -- *too-high genstamp*: the replica being reported has a higher generation stamp than what we think is current for the block. Queue it. -- *correct genstamp, wrong state*: eg a FINALIZED replica for an incomplete block. Queue it. - When replaying edits, check the queue whenever (a) a new block is created, (b) a block's genstamp is updated, (c) a block's completion state is changed -- if the block has just become complete, process any FINALIZED reports -- if the block has just been allocated or gen-stamp-bumped, process any RBW reports - During a failover, after we have completely caught up our namespace state, process all pending messages regardless of whether they are consistent. This is kind of complicated, but I can't think of much better. The one nice advantage it brings is that we don't have to delay a large BR full of old blocks just because it happens to include just one new block. This should keep the standby hotter and avoid using a bunch of memory for queued messages. HA: observed dataloss in replication stress test Key: HDFS-2742 URL: https://issues.apache.org/jira/browse/HDFS-2742 Project: Hadoop HDFS Issue Type: Sub-task Components: data-node, ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Blocker Attachments: hdfs-2742.txt, log-colorized.txt The replication stress test case failed over the weekend since one of the replicas went missing. Still diagnosing the issue, but it seems like the chain of events was something like: - a block report was generated on one of the nodes while the block was being written - thus the block report listed the block as RBW - when the standby replayed this queued message, it was replayed after the file was marked complete. Thus it marked this replica as corrupt - it asked the DN holding the corrupt replica to delete it. And, I think, removed it from the block map at this time. - That DN then did another block report before receiving the deletion. This caused it to be re-added to the block map, since it was FINALIZED now. - Replication was lowered on the file, and it counted the above replica as non-corrupt, and asked for the other replicas to be deleted. - All replicas were lost. -- This message is automatically
[jira] [Commented] (HDFS-2742) HA: observed dataloss in replication stress test
[ https://issues.apache.org/jira/browse/HDFS-2742?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13182947#comment-13182947 ] Todd Lipcon commented on HDFS-2742: --- I'm looking into a simpler solution where we redo a bit of how the safe block count is tracked during safemode. This would allow 3 and 4 to be safely inverted above without breaking safemode, if I can get it to work right. HA: observed dataloss in replication stress test Key: HDFS-2742 URL: https://issues.apache.org/jira/browse/HDFS-2742 Project: Hadoop HDFS Issue Type: Sub-task Components: data-node, ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Blocker Attachments: hdfs-2742.txt, log-colorized.txt The replication stress test case failed over the weekend since one of the replicas went missing. Still diagnosing the issue, but it seems like the chain of events was something like: - a block report was generated on one of the nodes while the block was being written - thus the block report listed the block as RBW - when the standby replayed this queued message, it was replayed after the file was marked complete. Thus it marked this replica as corrupt - it asked the DN holding the corrupt replica to delete it. And, I think, removed it from the block map at this time. - That DN then did another block report before receiving the deletion. This caused it to be re-added to the block map, since it was FINALIZED now. - Replication was lowered on the file, and it counted the above replica as non-corrupt, and asked for the other replicas to be deleted. - All replicas were lost. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2766) HA: test for case where standby partially reads log and then performs checkpoint
[ https://issues.apache.org/jira/browse/HDFS-2766?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183047#comment-13183047 ] Todd Lipcon commented on HDFS-2766: --- Looks good. Can you add a javadoc to the new test case explaining what the potential bug it's looking for is (just copy-paste from the jira description)? Or point to HDFS-2766 in the javadoc? HA: test for case where standby partially reads log and then performs checkpoint Key: HDFS-2766 URL: https://issues.apache.org/jira/browse/HDFS-2766 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Attachments: HDFS-2766-HDFS-1623.patch Here's a potential bug case that we don't currently test for: - SBN is reading a finalized edits file when NFS disappears halfway through (or some intermittent error happens) - SBN performs a checkpoint and uploads it to the NN - NN receives a checkpoint that doesn't correspond to the end of any log segment - Both NN and SBN should be able to restart at this point. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2738) FSEditLog.selectinputStreams is reading through in-progress streams even when non-in-progress are requested
[ https://issues.apache.org/jira/browse/HDFS-2738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183050#comment-13183050 ] Todd Lipcon commented on HDFS-2738: --- {code} + throw new IOException(String.format(Gap in transactions, max txnid is %d + + , 0 txns from %d, toAtLeastTxId, fromTxId)); {code} I think we can improve this message. Perhaps something like: Gap in transactions. Expected to be able to read up until at least txid %d but unable to find any edit logs containing txid %d? Do you think that's clearer? {code} - inspector.getMaxSeenTxId(), - false); + editLog.isOpenForWrite() ? inspector.getMaxSeenTxId() : 0, false); {code} Worth a comment here explaining the logic behind this, since it's a bit subtle. {code} + if (inProgressOk || !elf.isInProgress()) { {code} I think this code would read better to invert the condition and drop a level of nesting: {code} if (elf.isInProgress() !inProgressOk) { continue; } ... {code} Otherwise looks good to me. I was able to verify TestBackupNode passes. Did you verify that the exceptions as described in this ticket are no longer present while the SBN tails a NN which is making constant edits? FSEditLog.selectinputStreams is reading through in-progress streams even when non-in-progress are requested --- Key: HDFS-2738 URL: https://issues.apache.org/jira/browse/HDFS-2738 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Blocker Attachments: HDFS-2738-HDFS-1623.patch, HDFS-2738-HDFS-1623.patch The new code in HDFS-1580 is causing an issue with selectInputStreams in the HA context. When the active is writing to the shared edits, selectInputStreams is called on the standby. This ends up calling {{journalSet.getInputStream}} but doesn't pass the {{inProgressOk=false}} flag. So, {{getInputStream}} ends up reading and validating the in-progress stream unnecessarily. Since the validation results are no longer properly cached, {{findMaxTransaction}} also re-validates the in-progress stream, and then breaks the corruption check in this code. The end result is a lot of errors like: 2011-12-30 16:45:02,521 ERROR namenode.FileJournalManager (FileJournalManager.java:getNumberOfTransactions(266)) - Gap in transactions, max txnid is 579, 0 txns from 578 2011-12-30 16:45:02,521 INFO ha.EditLogTailer (EditLogTailer.java:run(163)) - Got error, will try again. java.io.IOException: No non-corrupt logs for txid 578 at org.apache.hadoop.hdfs.server.namenode.JournalSet.getInputStream(JournalSet.java:229) at org.apache.hadoop.hdfs.server.namenode.FSEditLog.selectInputStreams(FSEditLog.java:1081) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.doTailEdits(EditLogTailer.java:115) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.access$0(EditLogTailer.java:100) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread.run(EditLogTailer.java:154) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2752) HA: exit if multiple shared dirs are configured
[ https://issues.apache.org/jira/browse/HDFS-2752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13183052#comment-13183052 ] Todd Lipcon commented on HDFS-2752: --- I agree with Eli - we don't currently use the JournalSet abstraction in EditLogTailer, so it can only use a single shared dir. Of course in the future we should support using multiple, but it adds some complexity to the initial release. HA: exit if multiple shared dirs are configured --- Key: HDFS-2752 URL: https://issues.apache.org/jira/browse/HDFS-2752 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Eli Collins We don't support multiple shared edits dirs, we should fail to start with an error in this case. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2762) TestCheckpoint is timing out
[ https://issues.apache.org/jira/browse/HDFS-2762?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13182356#comment-13182356 ] Todd Lipcon commented on HDFS-2762: --- +1, will commit momentarily after checking that the HA tests still pass. TestCheckpoint is timing out Key: HDFS-2762 URL: https://issues.apache.org/jira/browse/HDFS-2762 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Uma Maheswara Rao G Attachments: HDFS-2762.patch TestCheckpoint is timing out on the HA branch, and has been for a few days. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2762) TestCheckpoint is timing out
[ https://issues.apache.org/jira/browse/HDFS-2762?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13182360#comment-13182360 ] Todd Lipcon commented on HDFS-2762: --- It looks like TestStandbyCheckpoints is broken with this change in place, so not committing. Uma, can you take a look? TestCheckpoint is timing out Key: HDFS-2762 URL: https://issues.apache.org/jira/browse/HDFS-2762 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Uma Maheswara Rao G Attachments: HDFS-2762.patch TestCheckpoint is timing out on the HA branch, and has been for a few days. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2770) Block reports may mark corrupt blocks pending deletion as non-corrupt
[ https://issues.apache.org/jira/browse/HDFS-2770?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13182377#comment-13182377 ] Todd Lipcon commented on HDFS-2770: --- I believe the issue may be with any place we check: {code} // Ignore replicas already scheduled to be removed from the DN if(invalidateBlocks.contains(dn.getStorageID(), block)) { {code} since it is ignoring the fact that, after the replication monitor thread has run, the block is no longer in {{BlockManager.invalidateBlocks}}, but instead in that DatanodeDescriptor's {{invalidateBlocks}} list. Maybe someone can remind me why we even have two separate invalidateBlocks structures in the first place? (one global map keyed by StorageID and another per-datanode list) Block reports may mark corrupt blocks pending deletion as non-corrupt - Key: HDFS-2770 URL: https://issues.apache.org/jira/browse/HDFS-2770 Project: Hadoop HDFS Issue Type: Bug Components: name-node Affects Versions: 0.23.0 Reporter: Todd Lipcon Priority: Critical It seems like HDFS-900 may have regressed in trunk since it was committed without a regression test. In HDFS-2742 I saw the following sequence of events: - A block at replication 2 had one of its replicas marked as corrupt on the NN - NN scheduled deletion of that replica in {{invalidateWork}}, and removed it from the block map - The DN hosting that block sent a block report, which caused the replica to get re-added to the block map as if it were good - The deletion request was passed to the DN and it deleted the block - Now we're in a bad state, where the NN temporarily thinks that it has two good replicas, but in fact one of them has been deleted. If we lower replication of this block at this time, the one good remaining replica may be deleted. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2587) Add WebHDFS apt doc
[ https://issues.apache.org/jira/browse/HDFS-2587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13182385#comment-13182385 ] Todd Lipcon commented on HDFS-2587: --- Can anyone explain why these docs are in the hadoop-yarn site instead of a hadoop-hdfs site? I filed HDFS-2771 to move them, would appreciate your comments. Add WebHDFS apt doc --- Key: HDFS-2587 URL: https://issues.apache.org/jira/browse/HDFS-2587 Project: Hadoop HDFS Issue Type: Task Components: documentation Reporter: Tsz Wo (Nicholas), SZE Assignee: Tsz Wo (Nicholas), SZE Fix For: 0.24.0, 0.23.1 Attachments: h2587_2022.patch, h2587_2023.patch, h2587_2023b.patch, h2587_2023b_site.tar.gz, h2587_2023site.tar.gz This issue is to add a WebHDFS doc in apt format in additional to the forrest doc (HDFS-2552). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2756) Warm standby does not read the in_progress edit log
[ https://issues.apache.org/jira/browse/HDFS-2756?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181517#comment-13181517 ] Todd Lipcon commented on HDFS-2756: --- The old log files are purged when checkpoints are saved - see NNStorageRetentionManager Warm standby does not read the in_progress edit log Key: HDFS-2756 URL: https://issues.apache.org/jira/browse/HDFS-2756 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Hari Mankude Warm standby does not read the in_progress edit log. This could result in standby taking a long time to become the primary during a failover scenario. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2753) Standby namenode stuck in safemode during a failover
[ https://issues.apache.org/jira/browse/HDFS-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181521#comment-13181521 ] Todd Lipcon commented on HDFS-2753: --- This looks reasonable. Can you add a regression test to TestHASafeMode to demonstrate the problem before we commit it? Standby namenode stuck in safemode during a failover Key: HDFS-2753 URL: https://issues.apache.org/jira/browse/HDFS-2753 Project: Hadoop HDFS Issue Type: Bug Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Hari Mankude Assignee: Hari Mankude Attachments: HDFS-2753.patch Write traffic initiated from the client. Manual failover is done by killing NN and converting a different standby to active. NN is restarted as standby. The restarted standby stays in safemode forever. More information in the description. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2762) TestCheckpoint is timing out
[ https://issues.apache.org/jira/browse/HDFS-2762?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181574#comment-13181574 ] Todd Lipcon commented on HDFS-2762: --- It seems like the {{testMultipleSecondaryNameNodes}} test is the one that's timing out. I traced this to the commit of HDFS-2720. Uma, can you look into this? If we can't figure it out in the next day or so I'll revert HDFS-2720 on the branch to fix the tests. TestCheckpoint is timing out Key: HDFS-2762 URL: https://issues.apache.org/jira/browse/HDFS-2762 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Todd Lipcon TestCheckpoint is timing out on the HA branch, and has been for a few days. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181575#comment-13181575 ] Todd Lipcon commented on HDFS-2709: --- +1, looks good to me. Will commit momentarily after verifying the HA tests still pass on my machine. HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Attachments: HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2765) TestNameEditsConfigs is incorrectly swallowing IOE
[ https://issues.apache.org/jira/browse/HDFS-2765?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181726#comment-13181726 ] Todd Lipcon commented on HDFS-2765: --- +1 TestNameEditsConfigs is incorrectly swallowing IOE -- Key: HDFS-2765 URL: https://issues.apache.org/jira/browse/HDFS-2765 Project: Hadoop HDFS Issue Type: Bug Components: test Affects Versions: 0.24.0 Reporter: Aaron T. Myers Assignee: Aaron T. Myers Attachments: HDFS-2765.patch The final portion of this test case is swallowing an IOE and in so doing appearing to succeed, although it should not be succeeding as-written. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2737) HA: Automatically trigger log rolls periodically on the active NN
[ https://issues.apache.org/jira/browse/HDFS-2737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181727#comment-13181727 ] Todd Lipcon commented on HDFS-2737: --- bq. My preference is to read the editlog in progress and leave the rolling of editlog as it is today, a local NN decision That's not how it is today -- the SecondaryNameNode is the one that triggers it. bq. For HA requiring an external trigger for editlog rolling is going back to previous inflexibility It's not inflexible, since it's not tightly interlocked. The HA SBN may trigger a roll, but it's also fine if the NN rolls more often than this due to other triggers. HA: Automatically trigger log rolls periodically on the active NN - Key: HDFS-2737 URL: https://issues.apache.org/jira/browse/HDFS-2737 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Currently, the edit log tailing process can only read finalized log segments. So, if the active NN is not rolling its logs periodically, the SBN will lag a lot. This also causes many datanode messages to be queued up in the PendingDatanodeMessage structure. To combat this, the active NN needs to roll its logs periodically -- perhaps based on a time threshold, or perhaps based on a number of transactions. I'm not sure yet whether it's better to have the NN roll on its own or to have the SBN ask the active NN to roll its logs. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2764) HA: TestBackupNode is failing
[ https://issues.apache.org/jira/browse/HDFS-2764?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181786#comment-13181786 ] Todd Lipcon commented on HDFS-2764: --- Passes for me.. which revision are you testing from? HA: TestBackupNode is failing - Key: HDFS-2764 URL: https://issues.apache.org/jira/browse/HDFS-2764 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Looks like it has been for a few days. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2592) HA: Balancer support for HA namenodes
[ https://issues.apache.org/jira/browse/HDFS-2592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13180524#comment-13180524 ] Todd Lipcon commented on HDFS-2592: --- My hope is to propose a merge within the next week or so - most of the pieces for manual failover are done. The initial release would of course be considered alpha. HA: Balancer support for HA namenodes - Key: HDFS-2592 URL: https://issues.apache.org/jira/browse/HDFS-2592 Project: Hadoop HDFS Issue Type: Sub-task Components: balancer, ha Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Uma Maheswara Rao G The balancer currently interacts directly with namenode InetSocketAddresses and makes its own IPC proxies. We need to integrate it with HA so that it uses the same client failover infrastructure. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181031#comment-13181031 ] Todd Lipcon commented on HDFS-2709: --- I'm skeptical of the fix -- the question is _why_ we see the wrong log version here. We investigated and it looks like there's a race when a log file is created -- it preallocates the file with all 0x, and then it goes back and writes the version number. Adding a sleep() after the preallocate() call in EditLogFileOutputStream triggers this reliably. So, I think we should file another JIRA to fix that race. Separately, I agree that we should probably change this to be an exception instead of assert. But I think LogHeaderCorruptException is probably a better choice. HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Attachments: HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2756) Warm standby does not read the in_progress edit log
[ https://issues.apache.org/jira/browse/HDFS-2756?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=1318#comment-1318 ] Todd Lipcon commented on HDFS-2756: --- The FileJournalManager does support skipping to the middle of a segment once HDFS-2709 is committed (which should be tonight or tomorrow). Warm standby does not read the in_progress edit log Key: HDFS-2756 URL: https://issues.apache.org/jira/browse/HDFS-2756 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Hari Mankude Warm standby does not read the in_progress edit log. This could result in standby taking a long time to become the primary during a failover scenario. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181122#comment-13181122 ] Todd Lipcon commented on HDFS-2709: --- - Doesn't look like the interface audience annotation got added to EditLogInputException - tiny nit: extra space on this line: + streams = editLog.selectInputStreams(lastTxnId + 1, 0, false); - I don't believe that {{skipTrasnactions}} throws an IOE if you skip more than remain in the stream -- {{readOp}} will just return null, it seems, no? HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Attachments: HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2738) FSEditLog.selectinputStreams is reading through in-progress streams even when non-in-progress are requested
[ https://issues.apache.org/jira/browse/HDFS-2738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13181131#comment-13181131 ] Todd Lipcon commented on HDFS-2738: --- {code:title=BookKeeperJournalManager.java} + public EditLogInputStream getInputStream(long fromTxnId, boolean inProgessOk) + throws IOException { {code} typo in new parameter name. Same typo appears in a couple other places as well - {{grep -i}} the patch for it. - maybe add TODOs to handle this parameter in BK journal? Did you run the various tests that reference edit logs? I'd run {{TestOfflineEditsViewer,TestHDFSConcat,TestEditLogRace,TestNameEditsConfigs,TestSaveNamespace,TestEditLogFileOutputStream,TestFileJournalManager,TestCheckpoint,TestEditLog,TestFSEditLogLoader,TestFsLimits,TestSecurityTokenEditLog,TestStorageRestore,TestBackupNode,TestEditLogJournalFailures,TestEditLogTailer,TestEditLogsDuringFailover,TestHASafeMode,TestStandbyCheckpoints,TestDNFencing,TestDNFencingWithReplication,TestStandbyIsHot,TestGenericJournalConf,TestCheckPointForSecurityTokens,TestNNStorageRetentionManager,TestPBHelper,TestNNLeaseRecovery,TestFiRename}} FSEditLog.selectinputStreams is reading through in-progress streams even when non-in-progress are requested --- Key: HDFS-2738 URL: https://issues.apache.org/jira/browse/HDFS-2738 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Blocker Attachments: HDFS-2738-HDFS-1623.patch The new code in HDFS-1580 is causing an issue with selectInputStreams in the HA context. When the active is writing to the shared edits, selectInputStreams is called on the standby. This ends up calling {{journalSet.getInputStream}} but doesn't pass the {{inProgressOk=false}} flag. So, {{getInputStream}} ends up reading and validating the in-progress stream unnecessarily. Since the validation results are no longer properly cached, {{findMaxTransaction}} also re-validates the in-progress stream, and then breaks the corruption check in this code. The end result is a lot of errors like: 2011-12-30 16:45:02,521 ERROR namenode.FileJournalManager (FileJournalManager.java:getNumberOfTransactions(266)) - Gap in transactions, max txnid is 579, 0 txns from 578 2011-12-30 16:45:02,521 INFO ha.EditLogTailer (EditLogTailer.java:run(163)) - Got error, will try again. java.io.IOException: No non-corrupt logs for txid 578 at org.apache.hadoop.hdfs.server.namenode.JournalSet.getInputStream(JournalSet.java:229) at org.apache.hadoop.hdfs.server.namenode.FSEditLog.selectInputStreams(FSEditLog.java:1081) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.doTailEdits(EditLogTailer.java:115) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.access$0(EditLogTailer.java:100) at org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread.run(EditLogTailer.java:154) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2749) Wrong fsimage format while entering recovery mode
[ https://issues.apache.org/jira/browse/HDFS-2749?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13179614#comment-13179614 ] Todd Lipcon commented on HDFS-2749: --- Good find! I think this is probably the cause of HDFS-1029 which I saw in 2010. Do you have a unit test and/or patch for the problem? Wrong fsimage format while entering recovery mode - Key: HDFS-2749 URL: https://issues.apache.org/jira/browse/HDFS-2749 Project: Hadoop HDFS Issue Type: Bug Affects Versions: 0.20.2 Reporter: Denny Ye Priority: Critical Labels: hdfs hadoop is into a recovery mode and save namespace to disk before the system starting service. however, there are many situation will cause hadoop enter recovery mode like missing VERSION file and ckpt file exists due to last failure of checkpoint. in recovery mode, namespace is loaded from previous fsimage, and the default numFiles of namespace.rootDir is 1. the numFiles number is read from fsimage (readInt as version, readInt as namespaceId, readLong as numFiles). the numFiles number is not updated in namespace when saving namespace. save namespace just after load fsimage which actually write numFiles which is default value 1 to disk. the next time to load the saved fsimage from disk when rebooting or secondarynamenode doing checkpoint, the system will crash (OOM) because this fsimage is incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2291) HA: Checkpointing in an HA setup
[ https://issues.apache.org/jira/browse/HDFS-2291?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13179850#comment-13179850 ] Todd Lipcon commented on HDFS-2291: --- bq. dfs.namenode.standby.checkpoints - perhaps include .ha in there to make it clear that this option is only applicable in an HA setup renamed to dfs.ha.standby.checkpoints and DFS_HA_STANDBY_CHECKPOINTS_KEY {quote} Might as well make the members of CheckpointConf final. LOG.info(Counted txns in + file + : + val.getNumTransactions()); - Either should be removed or should not be info level. prepareStopStandbyServices is kind of a weird name. Perhaps prepareToStopStandbyServices ? // TODO interface audience in TransferFsImage TODO: need to cancel the savenamespace operation if it's in flight - I think this comment is no longer applicable to this patch, right? LOG.info(Time for a checkpoint !); - while strictly accurate, this doesn't seem to be the most helpful log message. e.printStackTrace(); in CheckpointerThread should probably be tossed. Nit: in CheckpointerThread#doWork: if(UserGroupInformation.isSecurityEnabled()) - space between if and (, and curly braces around body of if. You use System.currentTimeMillis in a bunch of places. How about replacing with o.a.h.hdfs.server.common.Util#now ? {quote} fixed the above bq. Does it not seem strange to you that the order of operations when setting a state is prepareExit - prepareEnter - exit - enter, instead of prepareExit - exit - prepareEnter - enter The point of the {{prepare*}} methods is that they have to happen before the lock is taken. So, {{prepareEnter}} can't happen after {{exit}}, because the lock already is held there. I clarified the javadoc a bit. bq. What's the point of the changes in EditLogTailer? In order for the test to spy on saveNamespace, I had to move the {{getFSImage}} call down. Otherwise, the spy wasn't getting picked up properly and the test was failing. bq. Can we make CheckpointerThread a static inner class? Currently it calls {{doCheckpoint}} in the outer class. I suppose it could be static, but it isn't really easy to test in isolation anyway, so I'm going to punt o this. bq. Does it make sense to explicitly disallow the SBN from allowing checkpoints to be uploaded to it? Yes and no... I sort of see your point. But, people have also discussed an external tool which would perform checkpoints for many clusters and then upload them HA: Checkpointing in an HA setup Key: HDFS-2291 URL: https://issues.apache.org/jira/browse/HDFS-2291 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Todd Lipcon Fix For: HA branch (HDFS-1623) Attachments: hdfs-2291.txt, hdfs-2291.txt, hdfs-2291.txt We obviously need to create checkpoints when HA is enabled. One thought is to use a third, dedicated checkpointing node in addition to the active and standby nodes. Another option would be to make the standby capable of also performing the function of checkpointing. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13179983#comment-13179983 ] Todd Lipcon commented on HDFS-2709: --- {code} + public void skipTransactions(long transactionsToSkip) throws IOException { +if (firstTxId != HdfsConstants.INVALID_TXID +lastTxId != HdfsConstants.INVALID_TXID) { + for (long i = 0; i transactionsToSkip; i++) { +reader.readOp(); + } +} {code} What's the if condition for? If those are pre-requisites for the function, shouldn't it be a Preconditions.checkState or something? What's the expected behavior if there are fewer than {{transactionsToSkip}} txns left in the file? Should specify in the javadoc. Also should add a checkArgument that transactionsToSkip = 0 I think. {code} +public class EditLogInputException extends IOException { {code} needs an audience annotation (or can it be made package-protected?) - In TestFileJournalManager, can we add a test which calls getInputStream() in the middle of a file and verifies that the first txn is the one we expect? - The modified TestHASafeMode never sets mockRuntime to override the actual Runtime object, so the verification is a no-op Otherwise looks good HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Attachments: HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13179989#comment-13179989 ] Todd Lipcon commented on HDFS-2709: --- Oh, I also had to add the following: {code} --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java @@ -180,6 +180,9 @@ public class EditLogTailer { while (shouldRun) { try { doTailEdits(); +} catch (InterruptedException ie) { + // interrupter should have already set shouldRun to false + continue; } catch (Throwable t) { LOG.error(Error encountered while tailing edits. Shutting down + standby NN., t); {code} or else the TestStandbyCheckpoints test was failing for me. HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Attachments: HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2751) Datanode drops OS cache behind reads even for short reads
[ https://issues.apache.org/jira/browse/HDFS-2751?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13179997#comment-13179997 ] Todd Lipcon commented on HDFS-2751: --- (thanks to JD Cryans for finding this one) Datanode drops OS cache behind reads even for short reads - Key: HDFS-2751 URL: https://issues.apache.org/jira/browse/HDFS-2751 Project: Hadoop HDFS Issue Type: Bug Components: data-node Affects Versions: 0.23.0, 0.24.0 Reporter: Todd Lipcon Assignee: Todd Lipcon HDFS-2465 has some code which attempts to disable the drop cache behind reads functionality when the reads are 256KB (eg HBase random access). But this check was missing in the {{close()}} function, so it always drops cache behind reads regardless of the size of the read. This hurts HBase random read performance when this patch is enabled. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2592) HA: Balancer support for HA namenodes
[ https://issues.apache.org/jira/browse/HDFS-2592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13180026#comment-13180026 ] Todd Lipcon commented on HDFS-2592: --- Hey Uma. Any progress on this? Would be nice to have the initial HA release support the balancer. HA: Balancer support for HA namenodes - Key: HDFS-2592 URL: https://issues.apache.org/jira/browse/HDFS-2592 Project: Hadoop HDFS Issue Type: Sub-task Components: balancer, ha Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Uma Maheswara Rao G The balancer currently interacts directly with namenode InetSocketAddresses and makes its own IPC proxies. We need to integrate it with HA so that it uses the same client failover infrastructure. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2185) HA: ZK-based FailoverController
[ https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13180049#comment-13180049 ] Todd Lipcon commented on HDFS-2185: --- Sure, that makes sense. I'm a little skeptical that the ZK library can be done well entirely in isolation of having something to plug it into... but if it can be, certainly would work. HA: ZK-based FailoverController --- Key: HDFS-2185 URL: https://issues.apache.org/jira/browse/HDFS-2185 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Todd Lipcon This jira is for a ZK-based FailoverController daemon. The FailoverController is a separate daemon from the NN that does the following: * Initiates leader election (via ZK) when necessary * Performs health monitoring (aka failure detection) * Performs fail-over (standby to active and active to standby transitions) * Heartbeats to ensure the liveness It should have the same/similar interface as the Linux HA RM to aid pluggability. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13180237#comment-13180237 ] Todd Lipcon commented on HDFS-2709: --- hrm... I was running the unit tests and it looks like TestStandbyIsHot did a Runtime.exit with the newest patch. Also this made me notice that that LOG.error should be LOG.fatal. Can you try looping that test and see if you can figure out what's up? HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Attachments: HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2743) Streamline usage of bookkeeper journal manager
[ https://issues.apache.org/jira/browse/HDFS-2743?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13179089#comment-13179089 ] Todd Lipcon commented on HDFS-2743: --- This doesn't look quite right. The jar-with-dependencies has too many dependencies, including junit and extra copies of some things that Hadoop already depends on (eg commons-math, commons-logging, log4j, etc). It should include only the minimal dependencies, and probably better off shading them as well IMO. Streamline usage of bookkeeper journal manager -- Key: HDFS-2743 URL: https://issues.apache.org/jira/browse/HDFS-2743 Project: Hadoop HDFS Issue Type: Improvement Reporter: Ivan Kelly Assignee: Ivan Kelly Fix For: 0.24.0 Attachments: HDFS-2743.diff The current method of installing bkjournal manager involves generating a tarball, and extracting it with special flags over the hdfs distribution. This is cumbersome and prone to being broken by other changes (see https://svn.apache.org/repos/asf/hadoop/common/trunk@1220940). I think a cleaner way to doing this is to generate a single jar that can be placed in the lib dir of hdfs. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2731) Autopopulate standby name dirs if they're empty
[ https://issues.apache.org/jira/browse/HDFS-2731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177732#comment-13177732 ] Todd Lipcon commented on HDFS-2731: --- We don't currently support using the 2NN with HA -- we'd need to improve it to upload the checkpointed images to both NNs. If we think this is critical we should file another JIRA for it... in HDFS-2291 I actually have added a check to the 2NN startup that makes it fail to start if HA is enabled. Autopopulate standby name dirs if they're empty --- Key: HDFS-2731 URL: https://issues.apache.org/jira/browse/HDFS-2731 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Eli Collins To setup a SBN we currently format the primary then manually copy the name dirs to the SBN. The SBN should do this automatically. Specifically, on NN startup, if HA with a shared edits dir is configured and populated, if the SBN has empty name dirs it should downloads the image and log from the primary (as an optimization it could copy the logs from the shared dir). If the other NN is still in standby then it should fail to start as it does currently. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177761#comment-13177761 ] Todd Lipcon commented on HDFS-2709: --- Aaron and I chatted offline about the above questions a little bit. We think the following is the best route forward: - Instead of adding a new constructor to ELFIS, add a new seekToTxnId method which FileJournalManager can call after constructing it. (the reasoning being that this is more similar to the normal Java FileInputStream which has a separate seek() call) - In FSEditLogLoader, we decided that the custom exception would make the most sense -- i.e wrap the {{readOp}} call in a {{try/catch}} which would rethrow the exception with some kind of new {{EditLogInputException}}. The new exception would also have a getter to determine how many txns were successfully applied prior to the error. This is similar to how InterruptedIOException works in the standard library. - Regarding tests, the suggestion was to add some new test cases to {{TestFileJournalManager}} to exercise the new code in {{selectInputStreams}}. HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Attachments: HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2736) HA: support separate SBN and 2NN?
[ https://issues.apache.org/jira/browse/HDFS-2736?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177791#comment-13177791 ] Todd Lipcon commented on HDFS-2736: --- bq. if we fail over to the SBN does it continue to checkpoint? If the only active NN comes back, it will start in standby mode and then do checkpoints against the other one. It's only if the former active remains dead for some time that we might have a problem. bq. If not the log grows unbounded until the old primary comes back, if so does that create performance problems since the primary wasn't previously checkpointing? Certainly the first startup after a long outage will be slower, since it has to replay a lot of transactions. But it's not any different than what happens today if the 2NN goes down for some length of time. Given that people currently run successfully with only a single 2NN, I don't think this is particularly high priority. Do you disagree? HA: support separate SBN and 2NN? - Key: HDFS-2736 URL: https://issues.apache.org/jira/browse/HDFS-2736 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins HDFS-2291 adds support for making the SBN capable of checkpointing, seems like we may also need to support the 2NN checkpointing as well. Eg if we fail over to the SBN does it continue to checkpoint? If not the log grows unbounded until the old primary comes back, if so does that create performance problems since the primary wasn't previously checkpointing? -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2736) HA: support 2NN with SBN
[ https://issues.apache.org/jira/browse/HDFS-2736?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177858#comment-13177858 ] Todd Lipcon commented on HDFS-2736: --- Agreed. Currently we also basically support multiple-standby. Even though it wasn't a goal of the implementation, it basically fell out for free. HA: support 2NN with SBN Key: HDFS-2736 URL: https://issues.apache.org/jira/browse/HDFS-2736 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins HDFS-2291 adds support for making the SBN capable of checkpointing, seems like we may also need to support the 2NN checkpointing as well. Eg if we fail over to the SBN does it continue to checkpoint? If not the log grows unbounded until the old primary comes back, if so does that create performance problems since the primary wasn't previously checkpointing? -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2737) HA: Automatically trigger log rolls periodically on the active NN
[ https://issues.apache.org/jira/browse/HDFS-2737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177866#comment-13177866 ] Todd Lipcon commented on HDFS-2737: --- A couple options here: *1) Add a thread to the NN which rolls periodically (based on time or # txns)* This would be advantageous if we had some use cases for keeping edit log segments short even absent HA. The only case Aaron and I could brainstorm would be for backups, where it's a little easier to backup a finalized file compared to a rolling one. But we can satisfy this easily by adding a command line tool to trigger a roll, which a backup script can use. So it's not super compelling. 2) Add a new thread to the SBN which makes an IPC to the active and asks it to roll periodically Advantage here is simplicity. 3) Add some code to the EditLogTailer thread in the SBN which makes a call to the active NN to trigger a roll when necessary (eg when the PendingDatanodeMessage queue is too large, or it's been too long since it has read any edits). Advantage here is that the real motivation for the rolls is the EditLogTailer itself. We want to keep lag low (for fast recovery) and also keep the pending datanode queue small (to fit within memory bounds). By putting the trigger here, we can directly inspect those two variables, and trigger rolls when necessary. So I'm thinking option 3 is the best. HA: Automatically trigger log rolls periodically on the active NN - Key: HDFS-2737 URL: https://issues.apache.org/jira/browse/HDFS-2737 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Currently, the edit log tailing process can only read finalized log segments. So, if the active NN is not rolling its logs periodically, the SBN will lag a lot. This also causes many datanode messages to be queued up in the PendingDatanodeMessage structure. To combat this, the active NN needs to roll its logs periodically -- perhaps based on a time threshold, or perhaps based on a number of transactions. I'm not sure yet whether it's better to have the NN roll on its own or to have the SBN ask the active NN to roll its logs. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2737) HA: Automatically trigger log rolls periodically on the active NN
[ https://issues.apache.org/jira/browse/HDFS-2737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177869#comment-13177869 ] Todd Lipcon commented on HDFS-2737: --- bq. Is it worth considering supporting tailing non-finalized logs? Worth considering, but as I understand from Ivan and Jitendra, BK doesn't support this functionality yet. Since it's simpler to just cause frequent rolls, we may as well do it this way and also solve the problem for BK at the same time, IMO. HA: Automatically trigger log rolls periodically on the active NN - Key: HDFS-2737 URL: https://issues.apache.org/jira/browse/HDFS-2737 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Currently, the edit log tailing process can only read finalized log segments. So, if the active NN is not rolling its logs periodically, the SBN will lag a lot. This also causes many datanode messages to be queued up in the PendingDatanodeMessage structure. To combat this, the active NN needs to roll its logs periodically -- perhaps based on a time threshold, or perhaps based on a number of transactions. I'm not sure yet whether it's better to have the NN roll on its own or to have the SBN ask the active NN to roll its logs. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2692) HA: Bugs related to failover from/into safe-mode
[ https://issues.apache.org/jira/browse/HDFS-2692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177367#comment-13177367 ] Todd Lipcon commented on HDFS-2692: --- bq. In FSEditLogLoader#loadFSEdits, should we really be unconditionally calling FSNamesystem#notifyGenStampUpdate in the finally block? What if an error occurs and maxGenStamp is never updated in FSEditLogLoader#loadEditRecords This should be OK -- we'll just call it with the argument 0, which won't cause any problem (0 is lower than any possible queued gen stamp) bq. sp. Initiatling in TestHASafeMode#testComplexFailoverIntoSafemode fixed bq. In FSNamesystem#notifyGenStampUpdate, could be a better log message, and the log level should probably not be info: LOG.info(= notified of genstamp update for: + gs); Fixed and changed to DEBUG level bq. Why is SafeModeInfo#doConsistencyCheck costly? It doesn't seem like it should be. If it's not in fact expensive, we might as well make it run regardless of whether or not asserts are enabled You're right that it's not super expensive, but this code gets called on every block being reported during startup, which is a fair amount.. so I chose to maintain the current behavior, of only running the checks when asserts are enabled. bq. Is there really no better way to check if assertions are enabled? Not that I've ever found! :( bq. seems like they should all be made member methods and moved to MiniDFSCluster... Also seems like TestEditLogTailer#waitForStandbyToCatchUp should be moved to MiniDFSCluster. I'd like to move a bunch of these methods into a new {{HATestUtil}} class... can I do that in a follow-up JIRA? Eli said: bq. Nice change and tests. Nit, I'd add a comment in TestHASafeMode#restartStandby where the safemode extension is set indicating the rationale, it looked like the asserts at the end were racy because I missed this Fixed HA: Bugs related to failover from/into safe-mode Key: HDFS-2692 URL: https://issues.apache.org/jira/browse/HDFS-2692 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Critical Attachments: hdfs-2692.txt, hdfs-2692.txt In testing I saw an AssertionError come up several times when I was trying to do failover between two NNs where one or the other was in safe-mode. Need to write some unit tests to try to trigger this -- hunch is it has something to do with the treatment of safe block count while tailing edits in safemode. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2720) HA : TestStandbyIsHot is failing while copying in_use.lock file from NN1 nameSpaceDirs to NN2 nameSpaceDirs
[ https://issues.apache.org/jira/browse/HDFS-2720?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177371#comment-13177371 ] Todd Lipcon commented on HDFS-2720: --- Small nits: {code} + // Now format 1st NN and copy the storage dirs to remaining all. {code} to remaining all seems like a typo. copy the storage directory from that node to the others. would be better. Also I think it's easier to read first than 1st {code} + //Start all Namenodes {code} add space after {{//}} - The change to remove setRpcEngine looks unrelated - that should get cleaned up in trunk so it doesn't present a merge issue in the branch. HA : TestStandbyIsHot is failing while copying in_use.lock file from NN1 nameSpaceDirs to NN2 nameSpaceDirs Key: HDFS-2720 URL: https://issues.apache.org/jira/browse/HDFS-2720 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, test Affects Versions: HA branch (HDFS-1623) Reporter: Uma Maheswara Rao G Assignee: Uma Maheswara Rao G Attachments: HDFS-2720.patch To maintain the clusterID same , we are copying the namespaceDirs from 1st NN to other NNs. While copying this files, in_use.lock file may not allow to copy in all the OSs since it has aquired the lock on it. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2720) HA : TestStandbyIsHot is failing while copying in_use.lock file from NN1 nameSpaceDirs to NN2 nameSpaceDirs
[ https://issues.apache.org/jira/browse/HDFS-2720?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177434#comment-13177434 ] Todd Lipcon commented on HDFS-2720: --- That would be a nice improvement... but I think it makes sense to do this small fix that Uma proposed so the tests run on Windows, and then do the standby initialize from remote active feature separately? HA : TestStandbyIsHot is failing while copying in_use.lock file from NN1 nameSpaceDirs to NN2 nameSpaceDirs Key: HDFS-2720 URL: https://issues.apache.org/jira/browse/HDFS-2720 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, test Affects Versions: HA branch (HDFS-1623) Reporter: Uma Maheswara Rao G Assignee: Uma Maheswara Rao G Attachments: HDFS-2720.patch To maintain the clusterID same , we are copying the namespaceDirs from 1st NN to other NNs. While copying this files, in_use.lock file may not allow to copy in all the OSs since it has aquired the lock on it. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2732) Add support for the standby in the bin scripts
[ https://issues.apache.org/jira/browse/HDFS-2732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177479#comment-13177479 ] Todd Lipcon commented on HDFS-2732: --- For me, start-dfs.sh actually already works, since it uses the GetConf tool which prints out all of the NN addresses in the cluster based on the configuration. Does it not work for you? Add support for the standby in the bin scripts -- Key: HDFS-2732 URL: https://issues.apache.org/jira/browse/HDFS-2732 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Eli Collins We need to update the bin scripts to support SBNs. Two ideas: Modify start-dfs.sh to start another copy of the NN if HA is configured. We could introduce a file similar to masters (2NN hosts) called standbys which lists the SBN hosts, and start-dfs.sh would automatically make the NN it starts active (and leave the NNs listed in standby as is). Or simpler, we could just provide a start-namenode.sh script that a user can run to start the SBN on another host themselves. The user would manually tell the other NN to be active via HAAdmin (or start-dfs.sh could do that automatically, ie assume the NN it starts should be the primary). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2731) Autopopulate standby name dirs if they're empty
[ https://issues.apache.org/jira/browse/HDFS-2731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177478#comment-13177478 ] Todd Lipcon commented on HDFS-2731: --- bq. as an optimization it could copy the logs from the shared dir I dont think it's necessarily an optimization - might actually be _easier_ to implement this way :) bq. If the other NN is still in standby then it should fail to start as it does currently Can you explain what you mean by this? Why not allow it to download the image from the other NN anyway? Autopopulate standby name dirs if they're empty --- Key: HDFS-2731 URL: https://issues.apache.org/jira/browse/HDFS-2731 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Eli Collins To setup a SBN we currently format the primary then manually copy the name dirs to the SBN. The SBN should do this automatically. Specifically, on NN startup, if HA with a shared edits dir is configured and populated, if the SBN has empty name dirs it should downloads the image and log from the primary (as an optimization it could copy the logs from the shared dir). If the other NN is still in standby then it should fail to start as it does currently. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2731) Autopopulate standby name dirs if they're empty
[ https://issues.apache.org/jira/browse/HDFS-2731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177488#comment-13177488 ] Todd Lipcon commented on HDFS-2731: --- The primary shouldn't be removing any old images unless it's taking checkpoints. But there won't be checkpoints if the standby isn't running yet (assuming the standby is the one doing checkpointing). So if we get the most recent image from the NN, then we should always have enough edits in the shared dir to roll forward from there. Autopopulate standby name dirs if they're empty --- Key: HDFS-2731 URL: https://issues.apache.org/jira/browse/HDFS-2731 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Eli Collins To setup a SBN we currently format the primary then manually copy the name dirs to the SBN. The SBN should do this automatically. Specifically, on NN startup, if HA with a shared edits dir is configured and populated, if the SBN has empty name dirs it should downloads the image and log from the primary (as an optimization it could copy the logs from the shared dir). If the other NN is still in standby then it should fail to start as it does currently. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13177540#comment-13177540 ] Todd Lipcon commented on HDFS-2709: --- A few thoughts on the overall approach: - Rather than modify EditLogFileInputStream to take a startTxId, why not do the skipping (what you call {{setInitialPosition}}) from the caller? ie modify {{FSEditLogLoader}} to skip the transactions that have already been replayed? The skipping code doesn't seem specific to the input stream itself. - I'm not convinced why we need to have the {{partialLoadOk}} flag in {{FSEditLogLoader}}. IMO if the log is truncated, it's still an error as far as the loader is concerned - we just want to let the caller continue from where the error occured. The only trick is how to go about getting the last successfully loaded txid out of the FSEditLogLoader in the error case -- I guess a member variable and a getter would work there? Do you think this ends up messier than the way you've done it? - Can we add some non-HA tests that exercise FileJournalManager/FSEditLogLoader's ability to start mid-stream? Not sure if that's feasible. HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Attachments: HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch, HDFS-2709-HDFS-1623.patch Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2709) HA: Appropriately handle error conditions in EditLogTailer
[ https://issues.apache.org/jira/browse/HDFS-2709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13176232#comment-13176232 ] Todd Lipcon commented on HDFS-2709: --- What about the case where the edit log is large enough that it doesn't fit in RAM? I suppose we can deal with this by setting the active NN to roll every N MB where N is fairly small? HA: Appropriately handle error conditions in EditLogTailer -- Key: HDFS-2709 URL: https://issues.apache.org/jira/browse/HDFS-2709 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Aaron T. Myers Priority: Critical Currently if the edit log tailer experiences an error replaying edits in the middle of a file, it will go back to retrying from the beginning of the file on the next tailing iteration. This is incorrect since many of the edits will have already been replayed, and not all edits are idempotent. Instead, we either need to (a) support reading from the middle of a finalized file (ie skip those edits already applied), or (b) abort the standby if it hits an error while tailing. If a isn't simple, let's do b for now and come back to 'a' later since this is a rare circumstance and better to abort than be incorrect. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2720) HA : TestStandbyIsHot is failing while copying in_use.lock file from NN1 nameSpaceDirs to NN2 nameSpaceDirs
[ https://issues.apache.org/jira/browse/HDFS-2720?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13175532#comment-13175532 ] Todd Lipcon commented on HDFS-2720: --- For testing on actual clusters, I've done this by shutting down the active NN, then just rsyncing the storage dir to the standby, then starting the standby. Your idea of skipping in_use.lock is one solution for MiniDFSCluster. The other solution would be to copy the storage dir to all the standbys before starting the first NN. But that might break addNameNode support in HA - maybe not a big deal since I don't think we use that in HA cluster tests at the moment. HA : TestStandbyIsHot is failing while copying in_use.lock file from NN1 nameSpaceDirs to NN2 nameSpaceDirs Key: HDFS-2720 URL: https://issues.apache.org/jira/browse/HDFS-2720 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, test Affects Versions: HA branch (HDFS-1623) Reporter: Uma Maheswara Rao G Assignee: Uma Maheswara Rao G To maintain the clusterID same , we are copying the namespaceDirs from 1st NN to other NNs. While copying this files, in_use.lock file may not allow to copy in all the OSs since it has aquired the lock on it. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2623) HA: Add test case for hot standby capability
[ https://issues.apache.org/jira/browse/HDFS-2623?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13174942#comment-13174942 ] Todd Lipcon commented on HDFS-2623: --- Hi Uma. Yes, it passes here on Linux... my guess is there's an issue related to Windows having different semantics for file locking... can you look into it? I have no Windows box to test on. HA: Add test case for hot standby capability Key: HDFS-2623 URL: https://issues.apache.org/jira/browse/HDFS-2623 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, test Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Fix For: HA branch (HDFS-1623) Attachments: hdfs-2623.txt Putting up a fairly simple test case I wrote that verifies that the standby is kept hot -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2716) HA: Configuration needs to allow different dfs.http.addresses for each HA NN
[ https://issues.apache.org/jira/browse/HDFS-2716?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13175097#comment-13175097 ] Todd Lipcon commented on HDFS-2716: --- The {{getInfoServer}} function in {{DFSUtil}} only tries appending the nameservice ID, and not the NN ID. So we just need a small change here and there to fix this. I'll put a patch up hopefully tomorrow (on vacation til next week) HA: Configuration needs to allow different dfs.http.addresses for each HA NN Key: HDFS-2716 URL: https://issues.apache.org/jira/browse/HDFS-2716 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Earlier on the HA branch we expanded the configuration so that different IPC addresses can be specified for each of the HA NNs in a cluster. But we didn't do this for the HTTP address. This has proved problematic while working on HDFS-2291 (checkpointing in HA). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2718) Optimize OP_ADD in edits loading
[ https://issues.apache.org/jira/browse/HDFS-2718?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13175114#comment-13175114 ] Todd Lipcon commented on HDFS-2718: --- This is very similar to HDFS-2602 - we should at least share code from the HA branch so we don't have a nasty conflict at merge time. Optimize OP_ADD in edits loading Key: HDFS-2718 URL: https://issues.apache.org/jira/browse/HDFS-2718 Project: Hadoop HDFS Issue Type: Bug Components: name-node Affects Versions: 0.22.0, 0.24.0, 1.0.0 Reporter: Konstantin Shvachko During loading the edits journal FSEditLog.loadEditRecords() processes OP_ADD inefficiently. It first removes the existing INodeFile from the directory tree, then adds it back as a regular INodeFile, and then replaces it with INodeFileUnderConstruction if files is not closed. This slows down edits loading. OP_ADD should be done in one shot and retain previously existing data. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2185) HA: ZK-based FailoverController
[ https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13174246#comment-13174246 ] Todd Lipcon commented on HDFS-2185: --- Yea, this is very similar to the leader election recipe - I planned to base the code somewhat on that code for best practices. But the major difference is that we need to do fencing as well, which requires that we leave a non-ephemeral node behind when our ephemeral node expires, so the new NN can fence the old. HA: ZK-based FailoverController --- Key: HDFS-2185 URL: https://issues.apache.org/jira/browse/HDFS-2185 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Todd Lipcon This jira is for a ZK-based FailoverController daemon. The FailoverController is a separate daemon from the NN that does the following: * Initiates leader election (via ZK) when necessary * Performs health monitoring (aka failure detection) * Performs fail-over (standby to active and active to standby transitions) * Heartbeats to ensure the liveness It should have the same/similar interface as the Linux HA RM to aid pluggability. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2185) HA: ZK-based FailoverController
[ https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13174263#comment-13174263 ] Todd Lipcon commented on HDFS-2185: --- Twitter's also got a nice library of ZK stuff. But I think copy-paste is probably easier so we can customize it to our needs and not have to pull in lots of transitive dependencies HA: ZK-based FailoverController --- Key: HDFS-2185 URL: https://issues.apache.org/jira/browse/HDFS-2185 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Todd Lipcon This jira is for a ZK-based FailoverController daemon. The FailoverController is a separate daemon from the NN that does the following: * Initiates leader election (via ZK) when necessary * Performs health monitoring (aka failure detection) * Performs fail-over (standby to active and active to standby transitions) * Heartbeats to ensure the liveness It should have the same/similar interface as the Linux HA RM to aid pluggability. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2185) HA: ZK-based FailoverController
[ https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13174338#comment-13174338 ] Todd Lipcon commented on HDFS-2185: --- Great, thanks for the link, Uma. I will be sure to take a look. My plan is to finish off the checkpointing work next (HDFS-2291) and then go into a testing cycle for manual failover to make sure everything's robust. Unless we have a robust functional manual failover, automatic failover is just going to add some complication. After we're reasonably confident in the manual operation, we can start in earnest on the ZK-based automatic work. Do you agree? (of course it's good to start discussing design for the automatic one in parallel) HA: ZK-based FailoverController --- Key: HDFS-2185 URL: https://issues.apache.org/jira/browse/HDFS-2185 Project: Hadoop HDFS Issue Type: Sub-task Components: ha Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Todd Lipcon This jira is for a ZK-based FailoverController daemon. The FailoverController is a separate daemon from the NN that does the following: * Initiates leader election (via ZK) when necessary * Performs health monitoring (aka failure detection) * Performs fail-over (standby to active and active to standby transitions) * Heartbeats to ensure the liveness It should have the same/similar interface as the Linux HA RM to aid pluggability. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2713) HA : An alternative approach to clients handling Namenode failover.
[ https://issues.apache.org/jira/browse/HDFS-2713?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13174373#comment-13174373 ] Todd Lipcon commented on HDFS-2713: --- IMO it seems preferable to enhance (or replace) the existing code rather than introduce a new option. There's no sense in supporting both if one has clear advantages. If it's easier to write as new code, though, we could implement it as a new provider, then remove the old one when it gets committed. HA : An alternative approach to clients handling Namenode failover. Key: HDFS-2713 URL: https://issues.apache.org/jira/browse/HDFS-2713 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, hdfs client Affects Versions: HA branch (HDFS-1623) Reporter: Uma Maheswara Rao G Assignee: Uma Maheswara Rao G This is the approach for client failover which we adopted when we developed HA for Hadoop. I would like to propose thia approach for others to review include in the HA implementation, if found useful. This is similar to the ConfiguredProxyProvider in the sense that the it takes the address of both the Namenodes as the input. The major differences I can see from the current implementation are 1) During failover, user threads can be controlled very accurately about *the time they wait for active namenode* to be available, awaiting the retry. Beyond this, the threads will not be made to wait; DFS Client will throw an Exception indicating that the operation has failed. 2) Failover happens in a seperate thread, not in the client application threads. The thread will keep trying to find the Active Namenode until it succeeds. 3) This also means that irrespective of whether the operation's RetryAction is RETRY_FAILOVER or FAIL, the user thread can trigger the client's failover. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2699) Store data and checksums together in block file
[ https://issues.apache.org/jira/browse/HDFS-2699?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13173421#comment-13173421 ] Todd Lipcon commented on HDFS-2699: --- We already basically inline them on the wire in 64K chunks. But yes, I agree that if we inlined a checksum every 4 bytes it would be expensive. This is one of the advantages of doing it at the HBase layer - we're already reading an HFile block at a time (~64kb) - so putting the checksums in the header of the block doesn't add any extra read ops or non-contiguous IO. Store data and checksums together in block file --- Key: HDFS-2699 URL: https://issues.apache.org/jira/browse/HDFS-2699 Project: Hadoop HDFS Issue Type: Improvement Reporter: dhruba borthakur Assignee: dhruba borthakur The current implementation of HDFS stores the data in one block file and the metadata(checksum) in another block file. This means that every read from HDFS actually consumes two disk iops, one to the datafile and one to the checksum file. This is a major problem for scaling HBase, because HBase is usually bottlenecked on the number of random disk iops that the storage-hardware offers. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2693) Synchronization issues around state transition
[ https://issues.apache.org/jira/browse/HDFS-2693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13173766#comment-13173766 ] Todd Lipcon commented on HDFS-2693: --- bq. Can't we nuke the new code in commitBlockSynchronization since we should never execute this, ie we don't need a TODO here right? I was only moving it in this patch, so I didn't want to make a call on whether it's needed. I just noticed, when moving it, that it would never run. So I wanted to leave the TODO as a reminder that we need to look at pipeline recovery more closely, and here's one area where we clearly haven't handled it quite right (or need test coverage) bq. Nit: HAContext needs javadocs for new methods. Fixed, uploading new patch momentarily Synchronization issues around state transition -- Key: HDFS-2693 URL: https://issues.apache.org/jira/browse/HDFS-2693 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Critical Attachments: hdfs-2693.txt, hdfs-2693.txt, hdfs-2693.txt, hdfs-2693.txt Currently when the NN changes state, it does so without synchronization. In particular, the state transition function does: (1) leave old state (2) change state variable (3) enter new state This means that the NN is marked as active before it has actually transitioned to active mode and opened its edit logs. This gives a window where write transactions can come in and the {{checkOperation}} allows them, but then they fail because the edit log is not yet opened. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2692) HA: Bugs related to failover from/into safe-mode
[ https://issues.apache.org/jira/browse/HDFS-2692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13173775#comment-13173775 ] Todd Lipcon commented on HDFS-2692: --- btw, this patch is on top of HDFS-2693 and HDFS-1972. It also seems to have some issues related to BackupNode and CheckpointNode tests - working through those now but the gist of it is still worth reviewing. HA: Bugs related to failover from/into safe-mode Key: HDFS-2692 URL: https://issues.apache.org/jira/browse/HDFS-2692 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Critical Attachments: hdfs-2692.txt In testing I saw an AssertionError come up several times when I was trying to do failover between two NNs where one or the other was in safe-mode. Need to write some unit tests to try to trigger this -- hunch is it has something to do with the treatment of safe block count while tailing edits in safemode. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2291) HA: Checkpointing in an HA setup
[ https://issues.apache.org/jira/browse/HDFS-2291?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13173895#comment-13173895 ] Todd Lipcon commented on HDFS-2291: --- I plan to start working on this tomorrow. My thinking is to have a checkpoint thread which wakes up on the checkpoint interval, stops the edit log tailer thread, enters safe mode, creates a checkpoint, and comes back out of safemode. If at any point the SB needs to process a failover, it will cancel the checkpoint (using the HDFS-2507 feature) and proceed as usual. The remaining question I've yet to figure out is whether it should (a) save the checkpoints into the shared edits directory, or (b) save in its own and then upload the checkpoints to the primary via HTTP just like the 2NN does today. b is probably preferable since the shared edits directory may in fact be BK or some other journal plugin in the future, whereas a would break the abstraction. If anyone has any strong opinions please shout now :) HA: Checkpointing in an HA setup Key: HDFS-2291 URL: https://issues.apache.org/jira/browse/HDFS-2291 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Todd Lipcon Fix For: HA branch (HDFS-1623) We obviously need to create checkpoints when HA is enabled. One thought is to use a third, dedicated checkpointing node in addition to the active and standby nodes. Another option would be to make the standby capable of also performing the function of checkpointing. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2701) Cleanup FS* processIOError methods
[ https://issues.apache.org/jira/browse/HDFS-2701?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172511#comment-13172511 ] Todd Lipcon commented on HDFS-2701: --- Ah, right. I was thinking about trunk wrt restoreFailedStorage. +1 then Cleanup FS* processIOError methods -- Key: HDFS-2701 URL: https://issues.apache.org/jira/browse/HDFS-2701 Project: Hadoop HDFS Issue Type: Improvement Components: name-node Affects Versions: 1.0.0 Reporter: Eli Collins Assignee: Eli Collins Attachments: hdfs-2701.txt, hdfs-2701.txt, hdfs-2701.txt, hdfs-2701.txt Let's rename the various processIOError methods to be more descriptive. The current code makes it difficult to identify and reason about bug fixes. While we're at it let's remove Fatal from the Unable to sync the edit log log since it's not actually a fatal error (this is confusing to users). And 2NN Checkpoint done should be info, not a warning (also confusing to users). Thanks to HDFS-1073 these issues don't exist on trunk or 23. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2702) A single failed name dir can cause the NN to exit
[ https://issues.apache.org/jira/browse/HDFS-2702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172513#comment-13172513 ] Todd Lipcon commented on HDFS-2702: --- Oh, right. duh :) Thanks, +1. A single failed name dir can cause the NN to exit -- Key: HDFS-2702 URL: https://issues.apache.org/jira/browse/HDFS-2702 Project: Hadoop HDFS Issue Type: Bug Affects Versions: 1.0.0 Reporter: Eli Collins Assignee: Eli Collins Priority: Critical Attachments: hdfs-2702.txt, hdfs-2702.txt, hdfs-2702.txt, hdfs-2702.txt There's a bug in FSEditLog#rollEditLog which results in the NN process exiting if a single name dir has failed. Here's the relevant code: {code} close() // So editStreams.size() is 0 foreach edits dir { .. eStream = new ... // Might get an IOE here editStreams.add(eStream); } catch (IOException ioe) { removeEditsForStorageDir(sd); // exits if editStreams.size() = 1 } {code} If we get an IOException before we've added two edits streams to the list we'll exit, eg if there's an error processing the 1st name dir we'll exit even if there are 4 valid name dirs. The fix is to move the checking out of removeEditsForStorageDir (nee processIOError) or modify it so it can be disabled in some cases, eg here where we don't yet know how many streams are valid. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2678) HA: When a FailoverProxyProvider is used, DFSClient should not retry connection ten times before failing over
[ https://issues.apache.org/jira/browse/HDFS-2678?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172597#comment-13172597 ] Todd Lipcon commented on HDFS-2678: --- +1 modulo one comment: should use IPC_CLIENT_CONNECT_MAX_RETRIES_KEY instead of hardcoding the string. I'll make this change and commit it (atm left for vacation this morning) HA: When a FailoverProxyProvider is used, DFSClient should not retry connection ten times before failing over - Key: HDFS-2678 URL: https://issues.apache.org/jira/browse/HDFS-2678 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, hdfs client Affects Versions: HA branch (HDFS-1623) Reporter: Aaron T. Myers Assignee: Aaron T. Myers Attachments: HDFS-2678.patch Even though the {{FailoverOnNetworkExceptionRetry}} retry policy tries to fail over immediately, in the event the active is down, o.a.h.ipc.Client (below the RetryPolicy) will retry the initial connection 10 times before bubbling up this exception to the RetryPolicy. This should be much quicker. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2702) A single failed name dir can cause the NN to exit
[ https://issues.apache.org/jira/browse/HDFS-2702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172612#comment-13172612 ] Todd Lipcon commented on HDFS-2702: --- I think the addendum is correct. I wish we had a fault injection point for each of these cases, but that's a larger amount of work... something like how we use ErrorSimulator for errors in checkpoints would be great. A single failed name dir can cause the NN to exit -- Key: HDFS-2702 URL: https://issues.apache.org/jira/browse/HDFS-2702 Project: Hadoop HDFS Issue Type: Bug Affects Versions: 1.0.0 Reporter: Eli Collins Assignee: Eli Collins Priority: Critical Attachments: hdfs-2702.txt, hdfs-2702.txt, hdfs-2702.txt, hdfs-2702.txt, hdfs-2702.txt There's a bug in FSEditLog#rollEditLog which results in the NN process exiting if a single name dir has failed. Here's the relevant code: {code} close() // So editStreams.size() is 0 foreach edits dir { .. eStream = new ... // Might get an IOE here editStreams.add(eStream); } catch (IOException ioe) { removeEditsForStorageDir(sd); // exits if editStreams.size() = 1 } {code} If we get an IOException before we've added two edits streams to the list we'll exit, eg if there's an error processing the 1st name dir we'll exit even if there are 4 valid name dirs. The fix is to move the checking out of removeEditsForStorageDir (nee processIOError) or modify it so it can be disabled in some cases, eg here where we don't yet know how many streams are valid. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2162) Merge NameNode roles into NodeType.
[ https://issues.apache.org/jira/browse/HDFS-2162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172614#comment-13172614 ] Todd Lipcon commented on HDFS-2162: --- Suresh, are you still planning on working on this? Merge NameNode roles into NodeType. --- Key: HDFS-2162 URL: https://issues.apache.org/jira/browse/HDFS-2162 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: 0.23.0 Reporter: Suresh Srinivas Assignee: Suresh Srinivas Priority: Minor Currently Namenode has {{NamenodeRole}} with roles NAMENODE, BACKUP and CHECKPOINT. {{NodeType}} has node types NAME_NODE and DATA_NODE. Merge NamenodeRole into NodeType. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2682) HA: When a FailoverProxyProvider is used, Client should not retry for 45 times(hard coded value) if it is timing out to connect to server.
[ https://issues.apache.org/jira/browse/HDFS-2682?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172650#comment-13172650 ] Todd Lipcon commented on HDFS-2682: --- +1, will commit momentarily HA: When a FailoverProxyProvider is used, Client should not retry for 45 times(hard coded value) if it is timing out to connect to server. Key: HDFS-2682 URL: https://issues.apache.org/jira/browse/HDFS-2682 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, hdfs client Affects Versions: HA branch (HDFS-1623) Reporter: Uma Maheswara Rao G Assignee: Uma Maheswara Rao G Attachments: HDFS-2682.patch If Clients are getting SocketTimeoutException, when it is trying to connect to Server, it will retry for 45 times, to rethrow the exception to RetryPolicy. I think we can make this 45 retry times to configurable and set it to lower value when FailoverProxyProvider is used. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-1972) HA: Datanode fencing mechanism
[ https://issues.apache.org/jira/browse/HDFS-1972?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172742#comment-13172742 ] Todd Lipcon commented on HDFS-1972: --- I think I will actually integrate HDFS-2603 into this, since the issues are tightly linked. (I have a new test case which illustrates the issue). I'll upload a new patch soon, but this version is still worth reviewing. HA: Datanode fencing mechanism -- Key: HDFS-1972 URL: https://issues.apache.org/jira/browse/HDFS-1972 Project: Hadoop HDFS Issue Type: Sub-task Components: data-node, ha, name-node Reporter: Suresh Srinivas Assignee: Todd Lipcon Attachments: hdfs-1972-v1.txt, hdfs-1972.txt, hdfs-1972.txt, hdfs-1972.txt In high availability setup, with an active and standby namenode, there is a possibility of two namenodes sending commands to the datanode. The datanode must honor commands from only the active namenode and reject the commands from standby, to prevent corruption. This invariant must be complied with during fail over and other states such as split brain. This jira addresses issues related to this, design of the solution and implementation. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2191) Move datanodeMap from FSNamesystem to DatanodeManager
[ https://issues.apache.org/jira/browse/HDFS-2191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172780#comment-13172780 ] Todd Lipcon commented on HDFS-2191: --- Hi Nicholas. After this patch, the block invalidate limit is no longer configurable. Was this intentional? We still have the config key in DFSConfigKeys, but it's unused. Move datanodeMap from FSNamesystem to DatanodeManager - Key: HDFS-2191 URL: https://issues.apache.org/jira/browse/HDFS-2191 Project: Hadoop HDFS Issue Type: Sub-task Components: name-node Reporter: Tsz Wo (Nicholas), SZE Assignee: Tsz Wo (Nicholas), SZE Fix For: 0.23.0 Attachments: h2191_20110723.patch, h2191_20110723b.patch, h2191_20110726.patch -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2693) Synchronization issues around state transition
[ https://issues.apache.org/jira/browse/HDFS-2693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172886#comment-13172886 ] Todd Lipcon commented on HDFS-2693: --- Found a nasty bug in the current patch: in {{getBlockLocationsUpdateTimes}} the {{checkOperation}} call has to move down inside {{try..catch}} or else calling this function on an NN in standby state will leak the FSN readlock, killing the SBN. I'll upload a new patch soon. Still worth reviewing the general patch since most of it seems to be working correctly. Synchronization issues around state transition -- Key: HDFS-2693 URL: https://issues.apache.org/jira/browse/HDFS-2693 Project: Hadoop HDFS Issue Type: Sub-task Components: ha, name-node Affects Versions: HA branch (HDFS-1623) Reporter: Todd Lipcon Assignee: Todd Lipcon Priority: Critical Attachments: hdfs-2693.txt Currently when the NN changes state, it does so without synchronization. In particular, the state transition function does: (1) leave old state (2) change state variable (3) enter new state This means that the NN is marked as active before it has actually transitioned to active mode and opened its edit logs. This gives a window where write transactions can come in and the {{checkOperation}} allows them, but then they fail because the edit log is not yet opened. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2699) Store data and checksums together in block file
[ https://issues.apache.org/jira/browse/HDFS-2699?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13171961#comment-13171961 ] Todd Lipcon commented on HDFS-2699: --- The idea of introducing the new format as a backward-compatible option sounds good to me. That's what we did for the CRC32C checksums - new files are written with that checksum algorithm but old files continue to operate with the old one. Store data and checksums together in block file --- Key: HDFS-2699 URL: https://issues.apache.org/jira/browse/HDFS-2699 Project: Hadoop HDFS Issue Type: Improvement Reporter: dhruba borthakur Assignee: dhruba borthakur The current implementation of HDFS stores the data in one block file and the metadata(checksum) in another block file. This means that every read from HDFS actually consumes two disk iops, one to the datafile and one to the checksum file. This is a major problem for scaling HBase, because HBase is usually bottlenecked on the number of random disk iops that the storage-hardware offers. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2699) Store data and checksums together in block file
[ https://issues.apache.org/jira/browse/HDFS-2699?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172006#comment-13172006 ] Todd Lipcon commented on HDFS-2699: --- bq. Modifying any portion of that region will require that the entire data for the region be read in, and the CRC recomputed for that entire region and the entire region written out again But the cost of random-reading 4K is essentially the same as the cost of reading 512 bytes. Once you seek to the offset, the data transfer time is insignificant. Plus, given the 4KB page size used by Linux, all IO is already at this granularity. bq. An append happens a few days later to extend the file from 9K to 11K. CRC3 is now recomputed for the 3K-sized region spanning offsets 8K-11K and written out as CRC3-new. But there is a crash... This is an existing issue regardless of whether the checksums are interleaved or separate. The current solution is that we allow a checksum error on the last checksum chunk of a file in the case that it's being recovered after a crash -- iirc only in the case that _all_ replicas have this issue. If there is any valid replica, then we use that and truncate/rollback the other files to the sync boundary. Store data and checksums together in block file --- Key: HDFS-2699 URL: https://issues.apache.org/jira/browse/HDFS-2699 Project: Hadoop HDFS Issue Type: Improvement Reporter: dhruba borthakur Assignee: dhruba borthakur The current implementation of HDFS stores the data in one block file and the metadata(checksum) in another block file. This means that every read from HDFS actually consumes two disk iops, one to the datafile and one to the checksum file. This is a major problem for scaling HBase, because HBase is usually bottlenecked on the number of random disk iops that the storage-hardware offers. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2701) Cleanup FS* processIOError methods
[ https://issues.apache.org/jira/browse/HDFS-2701?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172023#comment-13172023 ] Todd Lipcon commented on HDFS-2701: --- in open(), if all of them fail to open, we'll have no edits streams... is that taken care of by 2702? in removeEditsForStorageDir, I think there might be a bug with the following sequence: - dir holding both edits and image fails - restoreFailedStorage is called so it is added back to the list for image operations, but edit logs haven't rolled yet, so it's not in editStreams - it fails again, so removeEditsForStorageDir is called with a dir that doesn't have any open stream. In that case, exitIfInvalidStreams() would exit even though nothing is getting removed. I guess this is taken care of by HDFS-2702? If the answer to both of the above is yes, then +1 :) Cleanup FS* processIOError methods -- Key: HDFS-2701 URL: https://issues.apache.org/jira/browse/HDFS-2701 Project: Hadoop HDFS Issue Type: Improvement Components: name-node Affects Versions: 1.0.0 Reporter: Eli Collins Assignee: Eli Collins Attachments: hdfs-2701.txt, hdfs-2701.txt, hdfs-2701.txt, hdfs-2701.txt Let's rename the various processIOError methods to be more descriptive. The current code makes it difficult to identify and reason about bug fixes. While we're at it let's remove Fatal from the Unable to sync the edit log log since it's not actually a fatal error (this is confusing to users). And 2NN Checkpoint done should be info, not a warning (also confusing to users). Thanks to HDFS-1073 these issues don't exist on trunk or 23. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2703) removedStorageDirs is not updated everywhere we remove a storage dir
[ https://issues.apache.org/jira/browse/HDFS-2703?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172024#comment-13172024 ] Todd Lipcon commented on HDFS-2703: --- +1 removedStorageDirs is not updated everywhere we remove a storage dir Key: HDFS-2703 URL: https://issues.apache.org/jira/browse/HDFS-2703 Project: Hadoop HDFS Issue Type: Bug Components: name-node Affects Versions: 1.0.0 Reporter: Eli Collins Assignee: Eli Collins Attachments: hdfs-2703.txt There are a number of places (FSEditLog#open, purgeEditLog, and rollEditLog) where we remove a storage directory but don't add it to the removedStorageDirs list. This means a storage dir may have been removed but we don't see it in the log or Web UI. This doesn't affect trunk/23 since the code there is totally different. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2702) A single failed name dir can cause the NN to exit
[ https://issues.apache.org/jira/browse/HDFS-2702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172025#comment-13172025 ] Todd Lipcon commented on HDFS-2702: --- - in {{fatalExit}}, can you change it to: {code} FSNamesystem.LOG.faral(msg, new Exception(msg)); {code} so that we get a stacktrace in the logs? - in {{exitIfNoStreams}} use {{isEmpty}} instead of comparing {{size() == 0}} - rather than an {{if...throw AssertionError}} maybe just use the {{Preconditions.checkState}} function from guava? Or is guava not in branch-1 yet? (can't remember) - instead of calling {{exitIfNoStreams}} everywhere, maybe {{removeEditsForStorageDir}} can just call it whenever it removes one? Otherwise looks good. A single failed name dir can cause the NN to exit -- Key: HDFS-2702 URL: https://issues.apache.org/jira/browse/HDFS-2702 Project: Hadoop HDFS Issue Type: Bug Affects Versions: 1.0.0 Reporter: Eli Collins Assignee: Eli Collins Priority: Critical Attachments: hdfs-2702.txt, hdfs-2702.txt, hdfs-2702.txt There's a bug in FSEditLog#rollEditLog which results in the NN process exiting if a single name dir has failed. Here's the relevant code: {code} close() // So editStreams.size() is 0 foreach edits dir { .. eStream = new ... // Might get an IOE here editStreams.add(eStream); } catch (IOException ioe) { removeEditsForStorageDir(sd); // exits if editStreams.size() = 1 } {code} If we get an IOException before we've added two edits streams to the list we'll exit, eg if there's an error processing the 1st name dir we'll exit even if there are 4 valid name dirs. The fix is to move the checking out of removeEditsForStorageDir (nee processIOError) or modify it so it can be disabled in some cases, eg here where we don't yet know how many streams are valid. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (HDFS-2679) Add interface to query current state to HAServiceProtocol
[ https://issues.apache.org/jira/browse/HDFS-2679?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13172032#comment-13172032 ] Todd Lipcon commented on HDFS-2679: --- +1. I'll commit this momentarily Add interface to query current state to HAServiceProtocol -- Key: HDFS-2679 URL: https://issues.apache.org/jira/browse/HDFS-2679 Project: Hadoop HDFS Issue Type: Sub-task Affects Versions: HA branch (HDFS-1623) Reporter: Eli Collins Assignee: Eli Collins Attachments: hdfs-2679.txt, hdfs-2679.txt, hdfs-2679.txt, hdfs-2679.txt, hdfs-2679.txt Let's add an interface to HAServiceProtocol to query the current state of a NameNode for use by the the CLI (HAAdmin) and Web UI (HDFS-2677). This essentially makes the names active and standby from ACTIVE_STATE and STANDBY_STATE public interfaces, which IMO seems reasonable. Unlike the other APIs we should be able to use the interface even when HA is not enabled (as by default a non-HA NN is active). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira