[jira] [Commented] (HDFS-3092) Enable journal protocol based editlog streaming for standby namenode

2012-04-12 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3092?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13252733#comment-13252733
 ] 

Bikas Saha commented on HDFS-3092:
--

Combination of this and HDFS-3077 sound very much like ZAB with one difference 
- in the use of 2-phase commit for the write broadcast.

Lets say there is 1 active NN writing to a quorum set of journal daemons. This 
is the same as ZAB. Active NN writes edits and ZAB leader writes new states.

ZAB uses 2 phase commits (without abort) for each write while our design is 
getting away without it. I am wondering why we can get away with it.

My guess is that each follower in ZAB can also serve reads from clients. Hence, 
it cannot serve an update until it is guaranteed that a quorum of followers has 
agreed on that update. That is what 2 phase commit gives. 
In our case, the active NN is the only server for client reads. Hence, updates 
are not served to clients until a quorum acks back.

However, the above would break for us if the standby NN is using any journal 
daemon to refresh its state. Because, ideally, a journal node should not inform 
the standby about an update until the it knows that the update has been 
accepted by a quorum of journal daemons. That would require a 2 phase commit.
E.g. Standby NN3 reads the last edit written to JN1 by old active NN1, before 
NN1 realized that it has lost quorum to NN2 (by failing to write to JN2 and 
JN3).

Perhaps we can get away with this by using some assumptions on timeouts, or by 
additional constraints on the standby. Eg. that it only syncs with finalized 
edit segments.

If we say that the standby sync with only the finalized log segments in order 
to be safe from the above, then IMO, the tailing of the edits by the standby 
should not be done by the standby directly but via a journal daemon API for the 
standby. This JD API would ensure that only valid edits are being sent to the 
standby (edits from finalized segments or edits known to be safely committed to 
a quorum of journal daemons). This way the correctness of the journal protocol 
would remain inside it. Instead of leaking it into the standby by having the 
standby code remember rules for tailing edits.



 Enable journal protocol based editlog streaming for standby namenode
 

 Key: HDFS-3092
 URL: https://issues.apache.org/jira/browse/HDFS-3092
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: ha, name-node
Affects Versions: 0.24.0, 0.23.3
Reporter: Suresh Srinivas
Assignee: Suresh Srinivas
 Attachments: MultipleSharedJournals.pdf, MultipleSharedJournals.pdf, 
 MultipleSharedJournals.pdf


 Currently standby namenode relies on reading shared editlogs to stay current 
 with the active namenode, for namespace changes. BackupNode used streaming 
 edits from active namenode for doing the same. This jira is to explore using 
 journal protocol based editlog streams for the standby namenode. A daemon in 
 standby will get the editlogs from the active and write it to local edits. To 
 begin with, the existing standby mechanism of reading from a file, will 
 continue to be used, instead of from shared edits, from the local edits.

--
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-3092) Enable journal protocol based editlog streaming for standby namenode

2012-04-12 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3092?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13252895#comment-13252895
 ] 

Bikas Saha commented on HDFS-3092:
--

Todd, I was going to post this same comment on HDFS-3077 because I did not see 
2 phase commits mentioned there either even though the initial comments 
mentioned implementing ZAB. The doc did mention standby node tailing as a 
requirement but it wasn't clear how this is being achieved in the absence of a 
coordinated distributed commit.

bq. That's my plan in HDFS-3077, and in fact that's the current behavior of the 
SBN, even when operating on NFS.
Hari updated me offline that standby NN currently tails only the finalized 
edits. So this works fine. By 'my plan' are you referring to an API on the 
journal node to read latest edits that replaces the current standby NN tailing 
code?

 Enable journal protocol based editlog streaming for standby namenode
 

 Key: HDFS-3092
 URL: https://issues.apache.org/jira/browse/HDFS-3092
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: ha, name-node
Affects Versions: 0.24.0, 0.23.3
Reporter: Suresh Srinivas
Assignee: Suresh Srinivas
 Attachments: MultipleSharedJournals.pdf, MultipleSharedJournals.pdf, 
 MultipleSharedJournals.pdf


 Currently standby namenode relies on reading shared editlogs to stay current 
 with the active namenode, for namespace changes. BackupNode used streaming 
 edits from active namenode for doing the same. This jira is to explore using 
 journal protocol based editlog streams for the standby namenode. A daemon in 
 standby will get the editlogs from the active and write it to local edits. To 
 begin with, the existing standby mechanism of reading from a file, will 
 continue to be used, instead of from shared edits, from the local edits.

--
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-3206) oev: miscellaneous xml cleanups

2012-04-05 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13247544#comment-13247544
 ] 

Bikas Saha commented on HDFS-3206:
--

At a higher level, is it ok for no user owner to exist and only have a group 
owner?
The current XML code seems to imply that a user will always be there. If that 
is true, and this bug showed up then it implies some namenode code is broken 
because it allows ownership to be group only.

Aside from that, the XML serde code should not reflect namenode behavior. So 
the fix looks good.


 oev: miscellaneous xml cleanups
 ---

 Key: HDFS-3206
 URL: https://issues.apache.org/jira/browse/HDFS-3206
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Colin Patrick McCabe
Assignee: Colin Patrick McCabe
Priority: Minor
 Attachments: HDFS-3206.001.patch


 * SetOwner operations can change both the user and group which a file or 
 directory belongs to, or just one of those.  Currently, in the XML 
 serialization/deserialization code, we don't handle the case where just the 
 group is set, not the user.  We should handle this case.
 * consistently serialize generation stamp as GENSTAMP.

--
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-3212) Persist the epoch received by the JournalService

2012-04-05 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13247854#comment-13247854
 ] 

Bikas Saha commented on HDFS-3212:
--

I have been trying to read ZAB and re-read PAXOS before I make some comments on 
some of the epoch stuff.
At first glance, it seems to me that some of these operations need to be 
atomic. I havent caught up with HDFS-3077 but I remember Tod clarifying to an 
example of mine by saying that edit log segments are relevant in the context of 
an epoch. So 2 edit logs with same txid but can be differentiated using epochs. 
In that case, it makes sense tying the epoch to segment relation in the roll 
via 1 above. Because then creating a segment and attaching it to an epoch would 
be 1 operation to the extent rolling is 1 operation.
2. might be less optimal because now it consists of 2 operations. 1) rolling 
the log and creating a new segment 2) updating a metadata file.
However, my understanding of rolling might be incomplete. So please take this 
with the necessary pinch of salt :P

 Persist the epoch received by the JournalService
 

 Key: HDFS-3212
 URL: https://issues.apache.org/jira/browse/HDFS-3212
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: Shared journals (HDFS-3092)
Reporter: Suresh Srinivas

 epoch received over JournalProtocol should be persisted by JournalService.

--
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-3077) Quorum-based protocol for reading and writing edit logs

2012-04-03 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3077?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13245729#comment-13245729
 ] 

Bikas Saha commented on HDFS-3077:
--

Nice doc! Greatly sped up understanding the design instead of having to grok it 
from the patch :)

I think it will help clarify the doc, if you add the explanation for Hari's 
example. Even though epoch 2 is persisted on JN1, its last log segment is still 
tied to epoch 1 and it needs to sync its last log segment with JN2/JN3. Are you 
proposing that JN1 drop its last edits in progress and pick up the 
corresponding finalized segment from JN1/JN2. Or is it TBD? 

Btw, there is some new code here but there seems to be some code in existing NN 
that changes the sequential journal sync to parallel (based on reading your doc 
and not your patch). I am guessing there will be other significant changes 
going forward. Are you planning on committing this to a branch or directly to 
trunk?


 Quorum-based protocol for reading and writing edit logs
 ---

 Key: HDFS-3077
 URL: https://issues.apache.org/jira/browse/HDFS-3077
 Project: Hadoop HDFS
  Issue Type: New Feature
  Components: ha, name-node
Reporter: Todd Lipcon
Assignee: Todd Lipcon
 Attachments: hdfs-3077-partial.txt, qjournal-design.pdf


 Currently, one of the weak points of the HA design is that it relies on 
 shared storage such as an NFS filer for the shared edit log. One alternative 
 that has been proposed is to depend on BookKeeper, a ZooKeeper subproject 
 which provides a highly available replicated edit log on commodity hardware. 
 This JIRA is to implement another alternative, based on a quorum commit 
 protocol, integrated more tightly in HDFS and with the requirements driven 
 only by HDFS's needs rather than more generic use cases. More details to 
 follow.

--
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: HDFS portion of ZK-based FailoverController

2012-04-03 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13245739#comment-13245739
 ] 

Bikas Saha commented on HDFS-2185:
--

I think you are missing the failure arc when transitionToStandby is called in 
InElection state.

Is there any scope for admin operations in ZKFC. Will ZKFC receive and accept a 
signal (manual admin/auto machine reboot) to stop services? At that point, in 
InElection state, how will it know that it needs to send transitionToStandby or 
not (based on whether it is active or not)?


 HA: HDFS portion of ZK-based FailoverController
 ---

 Key: HDFS-2185
 URL: https://issues.apache.org/jira/browse/HDFS-2185
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: auto-failover, ha
Affects Versions: 0.24.0, 0.23.3
Reporter: Eli Collins
Assignee: Todd Lipcon
 Fix For: Auto failover (HDFS-3042)

 Attachments: Failover_Controller.jpg, hdfs-2185.txt, hdfs-2185.txt, 
 hdfs-2185.txt, hdfs-2185.txt, hdfs-2185.txt, zkfc-design.pdf, 
 zkfc-design.pdf, zkfc-design.pdf, zkfc-design.tex


 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-3077) Quorum-based protocol for reading and writing edit logs

2012-04-03 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3077?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13245752#comment-13245752
 ] 

Bikas Saha commented on HDFS-3077:
--

bq. Nope, the thinking is that all of the new code will be encapsulated by 
QuorumJournalManager. So, from the NN's perspective, there is only a single 
edit log. It happens that that edit log is distributed and fault-tolerant 
underneath, but the NN would see it as a single required journal, and crash 
if it fails to sync.
Got it. So local edits and remote edits would be replaced by a single 
qjournaledits.


 Quorum-based protocol for reading and writing edit logs
 ---

 Key: HDFS-3077
 URL: https://issues.apache.org/jira/browse/HDFS-3077
 Project: Hadoop HDFS
  Issue Type: New Feature
  Components: ha, name-node
Reporter: Todd Lipcon
Assignee: Todd Lipcon
 Attachments: hdfs-3077-partial.txt, qjournal-design.pdf


 Currently, one of the weak points of the HA design is that it relies on 
 shared storage such as an NFS filer for the shared edit log. One alternative 
 that has been proposed is to depend on BookKeeper, a ZooKeeper subproject 
 which provides a highly available replicated edit log on commodity hardware. 
 This JIRA is to implement another alternative, based on a quorum commit 
 protocol, integrated more tightly in HDFS and with the requirements driven 
 only by HDFS's needs rather than more generic use cases. More details to 
 follow.

--
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-3190) Simple refactors in existing NN code to assist QuorumJournalManager extension

2012-04-03 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13245765#comment-13245765
 ] 

Bikas Saha commented on HDFS-3190:
--

+1 lgtm.


 Simple refactors in existing NN code to assist QuorumJournalManager extension
 -

 Key: HDFS-3190
 URL: https://issues.apache.org/jira/browse/HDFS-3190
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: name-node
Affects Versions: 2.0.0
Reporter: Todd Lipcon
Assignee: Todd Lipcon
Priority: Minor
 Attachments: hdfs-3190.txt


 This JIRA is for some simple refactors in the NN:
 - refactor the code which writes the seen_txid file in NNStorage into a new 
 LongContainingFile utility class. This is useful for the JournalNode to 
 atomically/durably record its last promised epoch
 - refactor the interface from FileJournalManager back to StorageDirectory to 
 use a StorageErrorReport interface. This allows FileJournalManager to be used 
 in isolation of a full StorageDirectory.

--
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-3077) Quorum-based protocol for reading and writing edit logs

2012-04-03 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3077?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13245785#comment-13245785
 ] 

Bikas Saha commented on HDFS-3077:
--

I have a question around syncing journal nodes and quorum based writes. There 
will always be a case that a lost journal node comes back up and is syncing its 
state - the extreme example of which is replacement of a broken journal node 
with a new node.
While it is doing this, will it be part of the quorum when a quorum number of 
writes must succeed?
Say we have 3 journals with the following txids
JN1-100, JN2-100, JN3-0 (JN3 just joined)
Now say some stuff got written to JN2 and JN3 (quorum commit with JN1 in flight 
records in the queue because JN1 is slow)
JN1-100, JN2-110, JN3-110+syncing_holes
At this point something terrible happens and when we recover, we can only 
access JN1 and JN3
JN1-100, JN3-110+syncing holes
At this point of time how do we resolve the ground truth about the journal 
state and edit logs?




 Quorum-based protocol for reading and writing edit logs
 ---

 Key: HDFS-3077
 URL: https://issues.apache.org/jira/browse/HDFS-3077
 Project: Hadoop HDFS
  Issue Type: New Feature
  Components: ha, name-node
Reporter: Todd Lipcon
Assignee: Todd Lipcon
 Attachments: hdfs-3077-partial.txt, qjournal-design.pdf


 Currently, one of the weak points of the HA design is that it relies on 
 shared storage such as an NFS filer for the shared edit log. One alternative 
 that has been proposed is to depend on BookKeeper, a ZooKeeper subproject 
 which provides a highly available replicated edit log on commodity hardware. 
 This JIRA is to implement another alternative, based on a quorum commit 
 protocol, integrated more tightly in HDFS and with the requirements driven 
 only by HDFS's needs rather than more generic use cases. More details to 
 follow.

--
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-3077) Quorum-based protocol for reading and writing edit logs

2012-03-30 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3077?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13242659#comment-13242659
 ] 

Bikas Saha commented on HDFS-3077:
--

You might want to switch this jira to HDFS-3042 and take the discussion off 
trunk.

 Quorum-based protocol for reading and writing edit logs
 ---

 Key: HDFS-3077
 URL: https://issues.apache.org/jira/browse/HDFS-3077
 Project: Hadoop HDFS
  Issue Type: New Feature
  Components: ha, name-node
Reporter: Todd Lipcon
Assignee: Todd Lipcon

 Currently, one of the weak points of the HA design is that it relies on 
 shared storage such as an NFS filer for the shared edit log. One alternative 
 that has been proposed is to depend on BookKeeper, a ZooKeeper subproject 
 which provides a highly available replicated edit log on commodity hardware. 
 This JIRA is to implement another alternative, based on a quorum commit 
 protocol, integrated more tightly in HDFS and with the requirements driven 
 only by HDFS's needs rather than more generic use cases. More details to 
 follow.

--
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: HDFS portion of ZK-based FailoverController

2012-03-29 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13241500#comment-13241500
 ] 

Bikas Saha commented on HDFS-2185:
--

I am surprised that only an inElection state suffices instead of an 
inElectionActive and inElectionStandby. Arent there actions that need to be 
performed differently when the FC is active or standby?

There is no state transition dependent on the result of transitionToActive(). 
If transitionToActive on NN fails then FC should quitElection IMO. Currently, 
it quits election only on HM events. Same for transitionToStandby on NN. If 
that fails then should we not do something?

Which state is performing fencing? The state machine does not show fencing? Is 
it happening in the the HAService?

IMO - this state diagram has to be more clear about handling the 
success/failure of each operation. That is key to determining the robustness of 
FC. FC needs to be super robust by design right?

 HA: HDFS portion of ZK-based FailoverController
 ---

 Key: HDFS-2185
 URL: https://issues.apache.org/jira/browse/HDFS-2185
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: auto-failover, ha
Affects Versions: 0.24.0, 0.23.3
Reporter: Eli Collins
Assignee: Todd Lipcon
 Attachments: Failover_Controller.jpg, hdfs-2185.txt, hdfs-2185.txt, 
 hdfs-2185.txt, hdfs-2185.txt, zkfc-design.pdf, zkfc-design.pdf, 
 zkfc-design.tex


 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: HDFS portion of ZK-based FailoverController

2012-03-26 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13238629#comment-13238629
 ] 

Bikas Saha commented on HDFS-2185:
--

Thanks! I will look at the doc.
ramblingI agree that the best documentation is the code. But that is for the 
developers. Putting up a design document helps people not familiar with the 
code or who dont have the time to extract the design from the code, to 
contribute potentially valuable inputs.
This is specially true for a new piece of code like this one, where there is 
not much previous background to rely on.
Once this is committed, then the code would be the best documentation. Updating 
docs would be good but is usually not practical.
I have learned that the more I do to help my reviewers, the better reviews I 
get. E.g. if I were doing it, in this case, I would also include a diagram that 
briefly sketches the code/object organization/relationships with tips on where 
to start so that my review process is made easy./rambling

 HA: HDFS portion of 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
 Attachments: Failover_Controller.jpg, hdfs-2185.txt, hdfs-2185.txt, 
 zkfc-design.pdf


 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

2012-03-23 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13237351#comment-13237351
 ] 

Bikas Saha commented on HDFS-2185:
--

It would be really great if there is a design document posted that explains the 
details. Thats usually a lot easier to understand (aside of actual 
white-boarding :)) than real code. It helps in reading the code if the mental 
model of the design is made via a document. Specially since this is a new 
component altogether.



 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
 Attachments: Failover_Controller.jpg, hdfs-2185.txt


 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-3096) dfs.datanode.data.dir.perm is set to 755 instead of 700

2012-03-18 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-3096?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13232402#comment-13232402
 ] 

Bikas Saha commented on HDFS-3096:
--

In branch 0.23 the datanode, at startup, overrides actual data dir permissions 
with expected permissions from the conf, in case they differ. In 1.0 the 
datanode checks for these values to be the same and refuses to start if they 
are different. A fix needs to be made for this behavior too.
This code path in 0.23 has been refactored and so the change may not be a 
simple backport. Perhaps we could change the code in 1.0 to also override 
actual permissions with those from the config.

 dfs.datanode.data.dir.perm is set to 755 instead of 700
 ---

 Key: HDFS-3096
 URL: https://issues.apache.org/jira/browse/HDFS-3096
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: data-node
Affects Versions: 0.23.0, 1.0.0
Reporter: Bikas Saha
Assignee: Bikas Saha

 dfs.datanode.data.dir.perm is used by the datanode to set the permissions of 
 it data directories. This is set by default to 755 which gives read 
 permissions to everyone to that directory, opening up possibility of reading 
 the data blocks by anyone in a secure cluster. Admins can over-ride this 
 config but its sub-optimal practice for the default to be weak. IMO, the 
 default should be strong and the admins can relax it if necessary.
 The fix is to change default permissions to 700.

--
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

2012-03-02 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13221370#comment-13221370
 ] 

Bikas Saha commented on HDFS-2185:
--

I have attached a state diagram for some ideas I had on how this could work. 
Think of the rectangles as the primary states of the controller. The ovals are 
actions that need to be taken before changing states. The black arrows are 
results of those actions and the blue arrows are external events. The blue 
arrows are notifications that can be received from the ZK leader election 
library added in HADOOP-7992 and the health notifications from the 
HAServiceProtocol.
This expects one change in the HAServiceProtocol. That is to split 
becomeActive() into prepareToBecomeActive() and becomeActive(). 
prepareToBecomeActive() does the time consuming heavy lifting and the world 
might change by the time it completes. At that point, if the node is still the 
leader, it can quickly becomeActive(). Else it can becomeStandby().

 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
 Attachments: Failover_Controller.jpg


 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-2910) HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir is inaccessible during log roll

2012-02-16 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13209542#comment-13209542
 ] 

Bikas Saha commented on HDFS-2910:
--

Sure. How do I do that?

 HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir 
 is inaccessible during log roll
 ---

 Key: HDFS-2910
 URL: https://issues.apache.org/jira/browse/HDFS-2910
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2902) HA: Allow new (shared) edits log dir to be configured while NN is running

2012-02-16 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13209543#comment-13209543
 ] 

Bikas Saha commented on HDFS-2902:
--

Sounds ok.

 HA: Allow new (shared) edits log dir to be configured while NN is running
 -

 Key: HDFS-2902
 URL: https://issues.apache.org/jira/browse/HDFS-2902
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-15 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13208808#comment-13208808
 ] 

Bikas Saha commented on HDFS-2909:
--

ran all tests under hadoop-hdfs and they pass.

 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
 Attachments: HDFS-2909-HDFS-1623.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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-15 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13209006#comment-13209006
 ] 

Bikas Saha commented on HDFS-2909:
--

Thanks!

 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
 Attachments: HDFS-2909-HDFS-1623.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-2910) HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir is inaccessible during log roll

2012-02-15 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13209199#comment-13209199
 ] 

Bikas Saha commented on HDFS-2910:
--

Not an issue as long as the NN dies upon such errors (like it currently does). 
In case, that changes, this will need to be fixed because it does not work 
right now. So I did not close it. I tried to find a JIRA to make NN not die 
upon such errors but did not find one.

 HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir 
 is inaccessible during log roll
 ---

 Key: HDFS-2910
 URL: https://issues.apache.org/jira/browse/HDFS-2910
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2910) HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir is inaccessible during log roll

2012-02-15 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13209200#comment-13209200
 ] 

Bikas Saha commented on HDFS-2910:
--

Not an issue as long as the NN dies upon such errors (like it currently does). 
In case that changes, this will need to be fixed because it does not work right 
now. So I did not close it. I tried to find a JIRA to make NN not die upon such 
errors but did not find one. If that is never going to happen then we can close 
this.

 HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir 
 is inaccessible during log roll
 ---

 Key: HDFS-2910
 URL: https://issues.apache.org/jira/browse/HDFS-2910
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-10 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205932#comment-13205932
 ] 

Bikas Saha commented on HDFS-2909:
--

storage.writeTransactionIdFileToStorage() may or may not be called depending on 
whether IOException's are swallowed at some point in FSEditLog.rollEditLogs(). 
There does not seem to be a logic/requirement of when 
storage.writeTransactionIdFileToStorage() should or should not be called. 
Marking bad storage directories is a side effect of calling 
storage.writeTransactionIdFileToStorage().
Also, JournalSet works independent of storage directory state.

 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-10 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205933#comment-13205933
 ] 

Bikas Saha commented on HDFS-2909:
--

storage.writeTransactionIdFileToStorage() may or may not be called depending on 
whether IOException's are swallowed at some point in FSEditLog.rollEditLogs(). 
There does not seem to be a logic/requirement of when 
storage.writeTransactionIdFileToStorage() should or should not be called. 
Marking bad storage directories is a side effect of calling 
storage.writeTransactionIdFileToStorage().
Also, JournalSet works independent of storage directory state.

 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2781) Add client protocol and DFSadmin for command to restore failed storage

2012-02-09 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2781?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13204812#comment-13204812
 ] 

Bikas Saha commented on HDFS-2781:
--

Is this JIRA still valid? If I understand right, the premise was the the NN 
would fall into standby mode when the shared edits dir fails. After the shared 
edits dir is restored, the admin could use the command proposed in this JIRA to 
refresh the dirs.
But current policy is for the NN to shutdown on shared edits dir failure. When 
the dir is brought back online, then the NN will pick it up on being restarted.
When NN moves to active or standby states then the FSEditLog.journalSet is 
refreshed and will refresh the storage dirs upon next log roll (if the restore 
flag is set). Perhaps we are better off restoring directories as part of moving 
from active/standby states (when we re-init the JournalSet) instead of as an 
explicit command. Seems more natural and 1 less thing to do for the admin. 


 Add client protocol and DFSadmin for command to restore failed storage
 --

 Key: HDFS-2781
 URL: https://issues.apache.org/jira/browse/HDFS-2781
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha
Affects Versions: HA branch (HDFS-1623)
Reporter: Eli Collins
Assignee: Eli Collins

 Per HDFS-2769, it's important that an admin be able to ask the NN to try to 
 restore failed storage since we may drop into SM until the shared edits dir 
 is restored (w/o having to wait for the next checkpoint). There's currently 
 an API (and usage in DFSAdmin) to flip the flag indicating whether the NN 
 should try to restore failed storage but not that it should actually attempt 
 to do so. This jira is to add one. This is useful outside HA but doing as an 
 HDFS-1623 sub-task since it's motivated by 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-2781) Add client protocol and DFSadmin for command to restore failed storage

2012-02-09 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2781?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13204816#comment-13204816
 ] 

Bikas Saha commented on HDFS-2781:
--

Or perhaps storage dirs could restored when the dfsAdmin -restoreFailedStorage 
command sets the option to true (as part of the command).
This would handle the non-HA cases.

 Add client protocol and DFSadmin for command to restore failed storage
 --

 Key: HDFS-2781
 URL: https://issues.apache.org/jira/browse/HDFS-2781
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha
Affects Versions: HA branch (HDFS-1623)
Reporter: Eli Collins
Assignee: Eli Collins

 Per HDFS-2769, it's important that an admin be able to ask the NN to try to 
 restore failed storage since we may drop into SM until the shared edits dir 
 is restored (w/o having to wait for the next checkpoint). There's currently 
 an API (and usage in DFSAdmin) to flip the flag indicating whether the NN 
 should try to restore failed storage but not that it should actually attempt 
 to do so. This jira is to add one. This is useful outside HA but doing as an 
 HDFS-1623 sub-task since it's motivated by 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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-09 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13204821#comment-13204821
 ] 

Bikas Saha commented on HDFS-2909:
--

Yes. That is what I found when I ran the experiment myself. The NN can restart 
OK after this event.

bq.The requirement is that we don't log any actual edits to the new edit log 
until it's been recorded in all of the storage directories. 

Does this mean that the NN will stop further edits until that storage dir is 
restored? Or it will create a new edit log, record it in the remaining healthy 
dirs and go on from there?


 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2915) HA: TestFailureOfSharedDir.testFailureOfSharedDir() has race condition

2012-02-09 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13204916#comment-13204916
 ] 

Bikas Saha commented on HDFS-2915:
--

Let me address these in the patch for HDFS-2912 which modifies the same test.

 HA: TestFailureOfSharedDir.testFailureOfSharedDir() has race condition
 --

 Key: HDFS-2915
 URL: https://issues.apache.org/jira/browse/HDFS-2915
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
Priority: Minor
 Fix For: HA branch (HDFS-1623)

 Attachments: HDFS-2915.HDFS-1623.patch, HDFS-2915.HDFS-1623.patch


 The test deletes the shared edits dir to simulate a failure. Then it calls 
 rollEditLogs() to trigger the deleted dir to be used and fail with an 
 IOException. Unfortunately, deleting the shared dir can put the NN in safe 
 mode due to lack of space. This causes a SafeModeException to be thrown when 
 rollEditDirs() is called. This exception is caught as an IOException in the 
 test but the associated assert in the catch block fails.
 This always happens in the debugger because the delay in stepping through 
 causes the safe mode change to happen before rollEditLogs() gets called.

--
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-2781) Add client protocol and DFSadmin for command to restore failed storage

2012-02-09 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2781?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205052#comment-13205052
 ] 

Bikas Saha commented on HDFS-2781:
--

bq. friendly, eg if storage restoration is already enabled you mihht not think 
that you should try to enable it to get this side effect.
In that case, rolling logs will restore the directories just like it works as 
of now
HA imposes higher restrictions compared to what works as of now. So we might 
need to do special stuff for HA only. Which might be trying to restore failed 
directories in the process of transitioning to active (maybe also standby)
From what I read of the code, the standby doesnt seem to bother with setting 
failed directories since its operations are all read only. So there might be 
no need for the standby to shutdown gracefully.
If the active moves to SM because of a bad required directory then it should 
restore all required directories when it goes out of safe mode or else complain 
and stay in safe mode. All this should happen after the admin has done the 
necessary pre-requisites and issued a -safeMode leave command.
bq. There's some interaction with fencing, here, though... one likely reason 
that the NN will lose touch with the shared storage is that another node has 
requested that the NAS device fence the host. Then, after the failover, the 
administrator might unfence the host from the NAS, and we don't want the NN to 
automatically come back to life at this point.
Does the NN come back out of safemode automatically or only after an admin 
command?

 Add client protocol and DFSadmin for command to restore failed storage
 --

 Key: HDFS-2781
 URL: https://issues.apache.org/jira/browse/HDFS-2781
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha
Affects Versions: HA branch (HDFS-1623)
Reporter: Eli Collins
Assignee: Eli Collins

 Per HDFS-2769, it's important that an admin be able to ask the NN to try to 
 restore failed storage since we may drop into SM until the shared edits dir 
 is restored (w/o having to wait for the next checkpoint). There's currently 
 an API (and usage in DFSAdmin) to flip the flag indicating whether the NN 
 should try to restore failed storage but not that it should actually attempt 
 to do so. This jira is to add one. This is useful outside HA but doing as an 
 HDFS-1623 sub-task since it's motivated by 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-2912) HA: Namenode not shutting down when shared edits dir is inaccessible

2012-02-09 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205054#comment-13205054
 ] 

Bikas Saha commented on HDFS-2912:
--

Thanks guys!

 HA: Namenode not shutting down when shared edits dir is inaccessible
 

 Key: HDFS-2912
 URL: https://issues.apache.org/jira/browse/HDFS-2912
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
 Fix For: HA branch (HDFS-1623)

 Attachments: HDFS-2909.HDFS-1623.patch, HDFS-2912.HDFS-1623.patch, 
 HDFS-2912.HDFS-1623.patch


 When there is an error in shared edits dir then current policy requires the 
 active name node to abort and shutdown.
 Currently there is no way to shut down the name node and hence this does not 
 happen even after all journals have been aborted on error. In fact the name 
 node stays Active and also is not in safe mode. Ideally it should shut down, 
 or at least go into safe mode or standby mode.

--
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-2902) HA: Allow new edits log dir to be configured while NN is running

2012-02-09 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205095#comment-13205095
 ] 

Bikas Saha commented on HDFS-2902:
--

So to add a new edits log dir, the dir needs to be added to the NNStorage 
object and also add an edits journal to the to FSEditLogs.

 HA: Allow new edits log dir to be configured while NN is running
 

 Key: HDFS-2902
 URL: https://issues.apache.org/jira/browse/HDFS-2902
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2915) HA: TestFailureOfSharedDir.testFailureOfSharedDir() has race condition

2012-02-08 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203838#comment-13203838
 ] 

Bikas Saha commented on HDFS-2915:
--

I am planning to make the shared dir inaccessible by changing write permissions 
 (instead of deleting the directory). That gets rid all potential race 
conditions.

 HA: TestFailureOfSharedDir.testFailureOfSharedDir() has race condition
 --

 Key: HDFS-2915
 URL: https://issues.apache.org/jira/browse/HDFS-2915
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
Priority: Minor

 The test deletes the shared edits dir to simulate a failure. Then it calls 
 rollEditLogs() to trigger the deleted dir to be used and fail with an 
 IOException. Unfortunately, deleting the shared dir can put the NN in safe 
 mode due to lack of space. This causes a SafeModeException to be thrown when 
 rollEditDirs() is called. This exception is caught as an IOException in the 
 test but the associated assert in the catch block fails.
 This always happens in the debugger because the delay in stepping through 
 causes the safe mode change to happen before rollEditLogs() gets called.

--
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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13202644#comment-13202644
 ] 

Bikas Saha commented on HDFS-2909:
--

Repro steps
1) Start 2 NN's in active standby mode
2) Remove write permissions from shared edits dir
3) Upon log roll triggered by standby, the active gets error when finalizing 
the edit logs
4) The error exception is caught way up on the stack and error does not get 
reported against the bad shared edits dir

This happens because error reporting happens when FSImage.rollEditLogs() calls 
storage.writeTransactionIdFileToStorage() which is called after 
FSEDit.rollEditLogs(). The error in FSEdit.rollEditLogs() raises an exception 
that is not handled in FSImage.rollEditLogs() and hence 
storage.writeTransactionIdFileToStorage() does not get called and no error is 
reported. The bad directory continues to remain in FSImage.storage.

 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2910) Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir is inaccessible during log roll

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13202676#comment-13202676
 ] 

Bikas Saha commented on HDFS-2910:
--

Repro steps
1) Start 2 NN's in active standby mode
2) Remove write permissions from shared edits dir
3) Upon log roll triggered by standby, the active gets error when finalizing 
the edit logs
4) FSEditLog.rollEditLogs() call endCurrentLogSegment() that hits exception. 
This exception is silently ignored assuming logSync will fix it later on
5) FSEditLog.rollEditLogs() now calls startLogSegment() that again hits 
exception. This exception is not silently ignored but is caught way up the 
stack.
6) After that shared edits dir comes back online by giving write permissions
7) Standby triggers log roll again. But active NN keeps complaining about 
BETWEEN_LOG_SEGMENTS with the following exception reported on the standby
12/02/07 00:46:55 INFO ha.EditLogTailer: Triggering log roll on remote NameNode 
localhost/127.0.0.1:28000
12/02/07 00:47:53 WARN ha.EditLogTailer: Unable to trigger a roll of the active 
NN
java.lang.IllegalStateException: java.lang.IllegalStateException: Bad state: 
BETWEEN_LOG_SEGMENTS
at 
com.google.common.base.Preconditions.checkState(Preconditions.java:172)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:887)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:831)
at 
org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:975)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:4026)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:741)
at 
org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:116)
at 
org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:8068)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:439)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:878)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1608)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1604)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1177)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1602)

at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:203)
at $Proxy11.rollEditLog(Unknown Source)
at 
org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB.rollEditLog(NamenodeProtocolTranslatorPB.java:162)
at 
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.triggerActiveLogRoll(EditLogTailer.java:256)
at 
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.access$3(EditLogTailer.java:253)
at 
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread.run(EditLogTailer.java:287)

 Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir is 
 inaccessible during log roll
 ---

 Key: HDFS-2910
 URL: https://issues.apache.org/jira/browse/HDFS-2910
 Project: Hadoop HDFS
  Issue Type: Sub-task
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13202846#comment-13202846
 ] 

Bikas Saha commented on HDFS-2909:
--

This is happening because JournalSet.mapJournalsAndReportErrors() calls 
abortAllJournals() and throws new IOException when a required journal fails (in 
this case, the shared dir). I still have to see why the NN continues to run as 
active after this.
Coming back to the above, it seems that the abortAllJournals() code implies 
that NN should stop running when something like this happens. That would mean 
that inaccessibility of the the single shared edits dir will cause the active 
NN to shutdown. Most likely the standby NN will also not be able to access the 
shared edits dir. Which means that the shared edits dir has become a single 
point of failure for the HA service.
Still looking at why NN did not abort.


 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13202884#comment-13202884
 ] 

Bikas Saha commented on HDFS-2909:
--

The NN did not abort because it simply threw an IOException after calling 
abortAllJournals(). The RPC server translated the IOException into a 
ServiceException and responded to the client. So the NN continued to run and 
also considers itself active.

 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2909) HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13202990#comment-13202990
 ] 

Bikas Saha commented on HDFS-2909:
--

Aside from all the above I see some other issues.
Say everything is healthy and FSImage.rollEditLogs() is called.
It first calls FSEditLogs.rollLogs that actually rolls the logs.
It then calls storage.writeTransactionIdFileToStorage() which records this in 
all storage dirs so that the information about the rolled edits is not lost.
However, NN could crash in after FSEditLogs.rollLogs() has completed and before 
storage.writeTransactionIdFileToStorage() is called. That might leave the data 
in an inconsistent state.

 HA: Inaccessible shared edits dir not getting removed from FSImage storage 
 dirs upon error
 --

 Key: HDFS-2909
 URL: https://issues.apache.org/jira/browse/HDFS-2909
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2910) HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir is inaccessible during log roll

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13202991#comment-13202991
 ] 

Bikas Saha commented on HDFS-2910:
--

I think FSEditLog should not be starting a new segment when ending the last one 
failed. Specifically in this case, the failure should abortAllJournals and 
shutdown the HA NN.
Even if we fix the NN shutdown case, this bug still needs to be fixed or else 
the edit logs will be left behind in an inconsistent state.



 HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir 
 is inaccessible during log roll
 ---

 Key: HDFS-2910
 URL: https://issues.apache.org/jira/browse/HDFS-2910
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2902) HA: Allow new shared edit logs dir to be configured while NN is running

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203002#comment-13203002
 ] 

Bikas Saha commented on HDFS-2902:
--

Reading the code shows a possible inconsistency issue.

FSImage.storage (an NNStorage object) manages the info about all storage dirs 
and records their health state. This includes edits and name dirs.
FSEditLogs.journalSet manages the info about all the journals and each journal 
maintains its own reference to the StorageDirectory it is writing to. This 
storage directory is managed by FSImage.storage above.

However, both these work independently. So marking a directory as bad in 
FSImage.storage does not really stop it from being written via a journal. And 
vice versa.



 HA: Allow new shared edit logs dir to be configured while NN is running
 ---

 Key: HDFS-2902
 URL: https://issues.apache.org/jira/browse/HDFS-2902
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2912) HA: Namenode not shutting down when shared edits dir is inaccessible

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203092#comment-13203092
 ] 

Bikas Saha commented on HDFS-2912:
--

For some reason the LOG.FATAL statements is not terminating the NN in my case. 
Will look into it further.

 HA: Namenode not shutting down when shared edits dir is inaccessible
 

 Key: HDFS-2912
 URL: https://issues.apache.org/jira/browse/HDFS-2912
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha

 When there is an error in shared edits dir then current policy requires the 
 active name node to abort and shutdown.
 Currently there is no way to shut down the name node and hence this does not 
 happen even after all journals have been aborted on error. In fact the name 
 node stays Active and also is not in safe mode. Ideally it should shut down, 
 or at least go into safe mode or standby mode.

--
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-2912) HA: Namenode not shutting down when shared edits dir is inaccessible

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203157#comment-13203157
 ] 

Bikas Saha commented on HDFS-2912:
--

From what I read of the code, for some of the cases (such as a flush of logs) 
where the NN actually dies on shared dir hiccups the runtime.exit() call was 
not added in the HA context. It was added when JournalSet was added by 
Jitendra long ago.

In any case, I would ideally like to have a cleaner shutdown mechanism to make 
sure that exit(1) do not proliferate in hard to find ways. Will let 
[HDFS-2913|https://issues.apache.org/jira/browse/HDFS-2913] track that.

For now, I will add an exit(1) after the LOG.FATAL in 
JournalSet.mapJournalsAndReportErrors(). This is the common code path through 
which all journal operations go through (roll edit logs, flush etc). So putting 
one here should hopefully catch all journal related cases.

 

 HA: Namenode not shutting down when shared edits dir is inaccessible
 

 Key: HDFS-2912
 URL: https://issues.apache.org/jira/browse/HDFS-2912
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha

 When there is an error in shared edits dir then current policy requires the 
 active name node to abort and shutdown.
 Currently there is no way to shut down the name node and hence this does not 
 happen even after all journals have been aborted on error. In fact the name 
 node stays Active and also is not in safe mode. Ideally it should shut down, 
 or at least go into safe mode or standby mode.

--
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-2910) HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir is inaccessible during log roll

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203160#comment-13203160
 ] 

Bikas Saha commented on HDFS-2910:
--

That is for the current policy of shutting down the NN on such errors. But if 
the NN continues to be active for short transient shared dir hiccups then this 
needs to be fixed. So I will let this JIRA remain active.

 HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir 
 is inaccessible during log roll
 ---

 Key: HDFS-2910
 URL: https://issues.apache.org/jira/browse/HDFS-2910
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2910) HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir is inaccessible during log roll

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203173#comment-13203173
 ] 

Bikas Saha commented on HDFS-2910:
--

Sure. Perhaps that work would resolve this JIRA too.

 HA: Active NN reports Bad state: BETWEEN_LOG_SEGMENTS when shared edits dir 
 is inaccessible during log roll
 ---

 Key: HDFS-2910
 URL: https://issues.apache.org/jira/browse/HDFS-2910
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2912) HA: Namenode not shutting down when shared edits dir is inaccessible

2012-02-07 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203210#comment-13203210
 ] 

Bikas Saha commented on HDFS-2912:
--

Could you please point me to the test that verifies the LOG.Fatal section that 
was added to JournalSet.mapJournalsAndReportErrors()?
I should ideally be modifying that test to verify the new change to that piece 
of code.


 HA: Namenode not shutting down when shared edits dir is inaccessible
 

 Key: HDFS-2912
 URL: https://issues.apache.org/jira/browse/HDFS-2912
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
 Attachments: HDFS-2909.HDFS-1623.patch


 When there is an error in shared edits dir then current policy requires the 
 active name node to abort and shutdown.
 Currently there is no way to shut down the name node and hence this does not 
 happen even after all journals have been aborted on error. In fact the name 
 node stays Active and also is not in safe mode. Ideally it should shut down, 
 or at least go into safe mode or standby mode.

--
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-2902) Allow new edit logs dir to be configured while NN is running

2012-02-06 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13201468#comment-13201468
 ] 

Bikas Saha commented on HDFS-2902:
--

While a NN is running the configured shared edit dir might fail and need to be 
replaced. When a new location is available then the admin can update the config 
and issue a refresh command to the NN. The NN will refresh the new edit dir and 
start using it the next time it rolls the edit logs.

 Allow new edit logs dir to be configured while NN is running
 

 Key: HDFS-2902
 URL: https://issues.apache.org/jira/browse/HDFS-2902
 Project: Hadoop HDFS
  Issue Type: Sub-task
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2902) Allow new shared edit logs dir to be configured while NN is running

2012-02-06 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13201626#comment-13201626
 ] 

Bikas Saha commented on HDFS-2902:
--

As I see the code now, it seems failed directories are restored upon rolling of 
edit logs. For 2781 are you suggesting the admin command triggers the 
restoration of the failed edits directories before the roll occurs?

 Allow new shared edit logs dir to be configured while NN is running
 ---

 Key: HDFS-2902
 URL: https://issues.apache.org/jira/browse/HDFS-2902
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2905) Standby NN NPE when shared edits dir is deleted

2012-02-06 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13201737#comment-13201737
 ] 

Bikas Saha commented on HDFS-2905:
--

Start 2 NN's with a shared edits dir. Make one active. Let some edits roll. Now 
delete the shared edits directory (simulating a shared edits failure). Standby 
NN NPE's with the following


12/02/06 14:41:27 WARN namenode.FSNamesystem: NameNode low on available disk 
space. Already in safe mode.
12/02/06 14:48:00 INFO hdfs.StateChange: STATE* Safe mode is ON. Resources are 
low on NN. Safe mode must be turned off manually.
12/02/06 14:48:05 WARN namenode.NameNodeResourceChecker: Space available on 
volume '/dev/disk0s2' is 0, which is below the configured reserved amount 
104857600
12/02/06 14:48:05 WARN namenode.FSNamesystem: NameNode low on available disk 
space. Already in safe mode.
12/02/06 14:48:05 INFO hdfs.StateChange: STATE* Safe mode is ON. Resources are 
low on NN. Safe mode must be turned off manually.
12/02/06 14:48:09 ERROR ha.EditLogTailer: Unknown error encountered while 
tailing edits. Shutting down standby NN.
java.lang.NullPointerException
at 
org.apache.hadoop.hdfs.server.namenode.FileJournalManager.matchEditLogs(FileJournalManager.java:160)
at 
org.apache.hadoop.hdfs.server.namenode.FileJournalManager.getLogFiles(FileJournalManager.java:321)
at 
org.apache.hadoop.hdfs.server.namenode.FileJournalManager.getNumberOfTransactions(FileJournalManager.java:228)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.getInputStream(JournalSet.java:216)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.selectInputStreams(FSEditLog.java:1074)
at 
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.doTailEdits(EditLogTailer.java:204)
at 
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.access$600(EditLogTailer.java:57)
at 
org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread.run(EditLogTailer.java:298)
12/02/06 14:48:09 INFO namenode.NameNode: SHUTDOWN_MSG: 
/


 Standby NN NPE when shared edits dir is deleted
 ---

 Key: HDFS-2905
 URL: https://issues.apache.org/jira/browse/HDFS-2905
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2781) Add client protocol and DFSadmin for command to restore failed storage

2012-02-06 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2781?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13201830#comment-13201830
 ] 

Bikas Saha commented on HDFS-2781:
--

I renamed the shared edits dir. The following happened
1) Active moved to safe mode. So it seems the above observation has already 
been fixed
2) Standby crashed with NPE 
[HDFS-2905|https://issues.apache.org/jira/browse/HDFS-2905]

Also, when the shared edits is brought back online (renaming it back) and the 
active is moved out of safe mode, then it starts re-using that directory when 
the standby rolls the edits.


 Add client protocol and DFSadmin for command to restore failed storage
 --

 Key: HDFS-2781
 URL: https://issues.apache.org/jira/browse/HDFS-2781
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha
Affects Versions: HA branch (HDFS-1623)
Reporter: Eli Collins
Assignee: Eli Collins

 Per HDFS-2769, it's important that an admin be able to ask the NN to try to 
 restore failed storage since we may drop into SM until the shared edits dir 
 is restored (w/o having to wait for the next checkpoint). There's currently 
 an API (and usage in DFSAdmin) to flip the flag indicating whether the NN 
 should try to restore failed storage but not that it should actually attempt 
 to do so. This jira is to add one. This is useful outside HA but doing as an 
 HDFS-1623 sub-task since it's motivated by 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-2781) Add client protocol and DFSadmin for command to restore failed storage

2012-02-06 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2781?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13201834#comment-13201834
 ] 

Bikas Saha commented on HDFS-2781:
--

Actually the active goes into safe mode on my machine because it thinks there 
is not enough space.

12/02/06 15:47:19 WARN namenode.FSNamesystem: NameNode low on available disk 
space. Already in safe mode.
12/02/06 15:47:19 INFO hdfs.StateChange: STATE* Safe mode is ON. Resources are 
low on NN. Safe mode must be turned off manually.
12/02/06 15:47:24 WARN namenode.NameNodeResourceChecker: Space available on 
volume '/dev/disk0s2' is 0, which is below the configured reserved amount 
104857600

So it might be that if the space constraint is removed then it might abort 
differently.

 Add client protocol and DFSadmin for command to restore failed storage
 --

 Key: HDFS-2781
 URL: https://issues.apache.org/jira/browse/HDFS-2781
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha
Affects Versions: HA branch (HDFS-1623)
Reporter: Eli Collins
Assignee: Eli Collins

 Per HDFS-2769, it's important that an admin be able to ask the NN to try to 
 restore failed storage since we may drop into SM until the shared edits dir 
 is restored (w/o having to wait for the next checkpoint). There's currently 
 an API (and usage in DFSAdmin) to flip the flag indicating whether the NN 
 should try to restore failed storage but not that it should actually attempt 
 to do so. This jira is to add one. This is useful outside HA but doing as an 
 HDFS-1623 sub-task since it's motivated by 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-2781) Add client protocol and DFSadmin for command to restore failed storage

2012-02-06 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2781?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13201937#comment-13201937
 ] 

Bikas Saha commented on HDFS-2781:
--

When I break on that code line, the change to safe mode is being triggered by 
the NameNodeResourceChecker returning false for resources available.
So DF returning 0 is what is causing the safe mode transition to occur.

What do you mean by parse error? Are you suggesting that the check for 
available space be replaced by something else when the available space == 0. 
Something that will actually check if the directory exists or not?



 Add client protocol and DFSadmin for command to restore failed storage
 --

 Key: HDFS-2781
 URL: https://issues.apache.org/jira/browse/HDFS-2781
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha
Affects Versions: HA branch (HDFS-1623)
Reporter: Eli Collins
Assignee: Eli Collins

 Per HDFS-2769, it's important that an admin be able to ask the NN to try to 
 restore failed storage since we may drop into SM until the shared edits dir 
 is restored (w/o having to wait for the next checkpoint). There's currently 
 an API (and usage in DFSAdmin) to flip the flag indicating whether the NN 
 should try to restore failed storage but not that it should actually attempt 
 to do so. This jira is to add one. This is useful outside HA but doing as an 
 HDFS-1623 sub-task since it's motivated by 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-2863) Failures observed if dfs.edits.dir and shared.edits.dir have same directories.

2012-02-02 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13199316#comment-13199316
 ] 

Bikas Saha commented on HDFS-2863:
--

I did not assign editDirs = uniqueEditDirs because I did not want the Set 
semantics to leak into the Collection return type of that method.
These are small lists anyways and so I wasnt bothered about the perf.

I will look at the other suggestions starting with finding out what a Joiner is 
:P

 Failures observed if dfs.edits.dir and shared.edits.dir have same directories.
 --

 Key: HDFS-2863
 URL: https://issues.apache.org/jira/browse/HDFS-2863
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Jitendra Nath Pandey
Assignee: Bikas Saha
 Attachments: HDFS-2863.HDFS-1623.patch


 If same edits directory is configured in twice, both are treated 
 independently. Edit log roll is called on the same directory twice causing 
 exceptions.

--
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-2863) Failures observed if dfs.edits.dir and shared.edits.dir have same directories.

2012-02-02 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13199370#comment-13199370
 ] 

Bikas Saha commented on HDFS-2863:
--

1.Will do
2.I added the TestFSNamesystem to be a class for adding future unit tests for 
the same source class. The test is not really HA specific since the dups can 
occur in the edits.dirs config itself right?
3.The unit test tests what the function does. So IMO it makes sense the way its 
written currently. The fact that we currently support a single shared edits dir 
is not related to the test and should be ideally covered in a functional sanity 
test for the HA config feature right?

Will update the patch shortly.

 Failures observed if dfs.edits.dir and shared.edits.dir have same directories.
 --

 Key: HDFS-2863
 URL: https://issues.apache.org/jira/browse/HDFS-2863
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Jitendra Nath Pandey
Assignee: Bikas Saha
 Attachments: HDFS-2863.HDFS-1623.patch, HDFS-2863.HDFS-1623.patch


 If same edits directory is configured in twice, both are treated 
 independently. Edit log roll is called on the same directory twice causing 
 exceptions.

--
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-2863) Failures observed if dfs.edits.dir and shared.edits.dir have same directories.

2012-02-02 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13199384#comment-13199384
 ] 

Bikas Saha commented on HDFS-2863:
--

I think we are mixing unit tests and functional tests here. If 2752 is properly 
implemented then I dont see why it should break the behavior of the existing 
methods since it defines a policy on top of a mechanism and these classes 
implement the mechanism.

In any case, there is no harm done in splitting them and its trivial to do so. 
Will update shortly.


 Failures observed if dfs.edits.dir and shared.edits.dir have same directories.
 --

 Key: HDFS-2863
 URL: https://issues.apache.org/jira/browse/HDFS-2863
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Jitendra Nath Pandey
Assignee: Bikas Saha
 Attachments: HDFS-2863.HDFS-1623.patch, HDFS-2863.HDFS-1623.patch, 
 HDFS-2863.HDFS-1623.patch


 If same edits directory is configured in twice, both are treated 
 independently. Edit log roll is called on the same directory twice causing 
 exceptions.

--
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-2863) Failures observed if dfs.edits.dir and shared.edits.dir have same directories.

2012-02-02 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13199455#comment-13199455
 ] 

Bikas Saha commented on HDFS-2863:
--

Thanks!

 Failures observed if dfs.edits.dir and shared.edits.dir have same directories.
 --

 Key: HDFS-2863
 URL: https://issues.apache.org/jira/browse/HDFS-2863
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Jitendra Nath Pandey
Assignee: Bikas Saha
 Attachments: HDFS-2863.HDFS-1623.patch, HDFS-2863.HDFS-1623.patch, 
 HDFS-2863.HDFS-1623.patch, HDFS-2863.HDFS-1623.patch, 
 HDFS-2863.HDFS-1623.patch


 If same edits directory is configured in twice, both are treated 
 independently. Edit log roll is called on the same directory twice causing 
 exceptions.

--
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-2845) SBN should not allow browsing of the file system via web UI

2012-02-01 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13198053#comment-13198053
 ] 

Bikas Saha commented on HDFS-2845:
--

Thanks guys!

 SBN should not allow browsing of the file system via web UI
 ---

 Key: HDFS-2845
 URL: https://issues.apache.org/jira/browse/HDFS-2845
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
 Fix For: HA branch (HDFS-1623)

 Attachments: HDFS-2485.HDFS-1623.patch, HDFS-2485.HDFS-1623.patch, 
 HDFS-2485.HDFS-1623.patch, HDFS-2485.HDFS-1623.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-2859) LOCAL_ADDRESS_MATCHER.match has NPE when called from DFSUtil.getSuffixIDs when the host is incorrect

2012-02-01 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2859?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13198373#comment-13198373
 ] 

Bikas Saha commented on HDFS-2859:
--

Thanks!

 LOCAL_ADDRESS_MATCHER.match has NPE when called from DFSUtil.getSuffixIDs 
 when the host is incorrect
 

 Key: HDFS-2859
 URL: https://issues.apache.org/jira/browse/HDFS-2859
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
Priority: Minor
 Fix For: HA branch (HDFS-1623)

 Attachments: HDFS-2859.HDFS-1623.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-2859) LOCAL_ADDRESS_MATCHER.match has NPE when called from DFSUtil.getSuffixIDs when the host is incorrect

2012-01-31 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2859?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13197339#comment-13197339
 ] 

Bikas Saha commented on HDFS-2859:
--

Based on the response to my question above I may resolve 
[HDFS-2858|https://issues.apache.org/jira/browse/HDFS-2858] with this patch. If 
its ok to have erroneous hosts then it might work to simply log the exception 
and proceed.

 LOCAL_ADDRESS_MATCHER.match has NPE when called from DFSUtil.getSuffixIDs 
 when the host is incorrect
 

 Key: HDFS-2859
 URL: https://issues.apache.org/jira/browse/HDFS-2859
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Bikas Saha
Assignee: Bikas Saha
Priority: Minor
 Attachments: HDFS-2859.HDFS-1623.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-2863) Failures observed if dfs.edits.dir and shared.edits.dir have same directories.

2012-01-31 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13197368#comment-13197368
 ] 

Bikas Saha commented on HDFS-2863:
--

Does it make sense to assert that the JournalSet must consists of distinct 
URI's as a policy decision. Then this check could be added to the JournalSet.


 Failures observed if dfs.edits.dir and shared.edits.dir have same directories.
 --

 Key: HDFS-2863
 URL: https://issues.apache.org/jira/browse/HDFS-2863
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Jitendra Nath Pandey
Assignee: Bikas Saha

 If same edits directory is configured in twice, both are treated 
 independently. Edit log roll is called on the same directory twice causing 
 exceptions.

--
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-2863) Failures observed if dfs.edits.dir and shared.edits.dir have same directories.

2012-01-31 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13197396#comment-13197396
 ] 

Bikas Saha commented on HDFS-2863:
--

Looks like it does not since the BackupJournal does not seem to be linked to a 
URI.

 Failures observed if dfs.edits.dir and shared.edits.dir have same directories.
 --

 Key: HDFS-2863
 URL: https://issues.apache.org/jira/browse/HDFS-2863
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: ha, name-node
Affects Versions: HA branch (HDFS-1623)
Reporter: Jitendra Nath Pandey
Assignee: Bikas Saha

 If same edits directory is configured in twice, both are treated 
 independently. Edit log roll is called on the same directory twice causing 
 exceptions.

--
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-2858) DFSUtil.getSuffixIDs silently ignores exception in NetUtils.createSocketAddr

2012-01-30 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13196332#comment-13196332
 ] 

Bikas Saha commented on HDFS-2858:
--

The catch block just has a continue and that makes debugging harder. I was 
trying to bring up an HA name node and getting errors. Did not know if there 
was an exception during socket creation. The exception trace would have helped.

 DFSUtil.getSuffixIDs silently ignores exception in NetUtils.createSocketAddr
 

 Key: HDFS-2858
 URL: https://issues.apache.org/jira/browse/HDFS-2858
 Project: Hadoop HDFS
  Issue Type: Sub-task
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2859) LOCAL_ADDRESS_MATCHER.match has NPE when called from DFSUtil.getSuffixIDs when the host is incorrect

2012-01-30 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2859?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13196333#comment-13196333
 ] 

Bikas Saha commented on HDFS-2859:
--

I was testing startup of an HA name node with an incorrect host and it crashed 
with an NPE here.

 LOCAL_ADDRESS_MATCHER.match has NPE when called from DFSUtil.getSuffixIDs 
 when the host is incorrect
 

 Key: HDFS-2859
 URL: https://issues.apache.org/jira/browse/HDFS-2859
 Project: Hadoop HDFS
  Issue Type: Sub-task
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2845) SBN should not allow browsing of the file system

2012-01-26 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13194069#comment-13194069
 ] 

Bikas Saha commented on HDFS-2845:
--

The state of the SBN is not up to date and hence it does not serve read 
requests from the client. As a corollary, it should not allow browsing of the 
file system from the web UI.

 SBN should not allow browsing of the file system
 

 Key: HDFS-2845
 URL: https://issues.apache.org/jira/browse/HDFS-2845
 Project: Hadoop HDFS
  Issue Type: Sub-task
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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-2845) SBN should not allow browsing of the file system via web UI

2012-01-26 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13194396#comment-13194396
 ] 

Bikas Saha commented on HDFS-2845:
--

The similarly named enums lead to confusion :P

 SBN should not allow browsing of the file system via web UI
 ---

 Key: HDFS-2845
 URL: https://issues.apache.org/jira/browse/HDFS-2845
 Project: Hadoop HDFS
  Issue Type: Sub-task
Reporter: Bikas Saha
Assignee: Bikas Saha



--
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

2012-01-16 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187149#comment-13187149
 ] 

Bikas Saha commented on HDFS-2681:
--

Can ActiveStandbyElector be made package-private?
Let me read up on all these package annotations and see what makes sense.

Also, it can't really guard against split brain
The guarantee is based on setting the correct timeouts. Another instance can 
only become active after the session timeout. The session timeout is 
recommended to be at least 3X the zookeeper disconnect timeout. 
enterNeutralMode is called when zookeeper client disconnects from zookeeper 
server or when zookeeper servers lose quorum. My understanding is that when 
there is a network disconnection then zookeeper client will disconnect from the 
server and post a disconnect event. 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. Does this clarify?

should also be ALL_CAPS
It public because its a well defined property of the class.
Is the ALLCAPS on static strings a convention? You mean the member name should 
be all caps or the value? I added the random UID to prevent accidental 
operation on this file from some admin. It does not hurt and it safer than 
using just a nicely named file. Anyways, I changed it.

Could use Arrays.copyOf()
I first used .clone() and then was pointed to System.ArrayCopy() and now 
pointed to Array.copyOf(). Could you please point me to any place that lists 
the pros and cons of different array copying methods (of which there seem to be 
many)?

Rename operationSuccess etc to isSuccessCode
I think the current names read OK with the if() statements.

Make ActiveStandbyElectorTester an inner class of TestActiveStandbyElector.
I first wrote it that way. But there is a problem. 
Tester_constructor()-super_constructor()-Tester().getNewZookeeper()-returns 
mock.
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.

Some of the INFO level logs are probably better off at DEBUG level.
Could you please point me to some place which explains what to log at different 
log levels?

 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-2681) Add ZK client for leader election

2012-01-16 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13187186#comment-13187186
 ] 

Bikas Saha commented on HDFS-2681:
--


About the GC pause scenario (and others like it). Lets not mix up election with 
operation safety. What this library provides is a signal about whether one is a 
leader or not. By itself, that does not solve the problems of whether that 
signal was properly processed or not. E.g. a potential solution to the GC pause 
(or any NN hung case) would be to not have the NN participate in leader 
election directly. A failover controller (whose design ensures 0 or cheap GC 
pauses) could handle the leader election and terminate hung NN's when its are 
no longer a master. 

Let me address some of the comments in a subsequent patch. I need to learn a 
little more Java before I can do it to my liking.


 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-2681) Add ZK client for leader election

2012-01-13 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13185757#comment-13185757
 ] 

Bikas Saha commented on HDFS-2681:
--

 General - How is multithreaded use for this library handled?
All public methods are synchronized

Add javadoc to the class on how to use the class. Also callback interface can 
be described in more detail on when the call back is made and perhaps some 
description of what is expected of the app. notifyError() particularly needs 
better documentation on when to expect this callback.
Please document the enumerations.
Done in second patch

Constructor should check for null - at least for call back passed. Otherwise 
you will get null pointer exception.
Done

joinElection() you may want to copy the byte[] data passed or at least 
document that the data[] must not be changed by the caller.
Done

#getNewZooKeeper() seems unnecessary and can be removed. Creation of 
ZooKeeper() can be moved to createConnection() it self.
This is to pass in a mock zookeeper for testing

Make member variable that are initialized only once in the constructor final.
Done in second patch

activeData could be better name for appData.
All app's can pass in data (which may go into future per app nodes). Only 
active app's data makes it to the lock. So I think the name is good.

Please check if all the params are documented in methods. For example 
constructor is missing one of the params in the doc. Same is true with 
exceptions thrown.
Done in second patch

quitElection() should not check zkClient non null, as terminateConnection 
already checks it.
Yeah. I forgot to remove that check after I refactored stuff into the reset() 
method

getActiveData() - how about not throwing KeeperException? Also 
ActiveNotFoundException should wrap the exception caught from ZK.
Its hard to differentiate exceptions inside KeeperException. There is not much 
the elector can do about them. The only commonly expected exception would be 
getting leader data when no leader exists and that has been handled as part of 
the elector API via a new exception.


 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, 
 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-2681) Add ZK client for leader election

2012-01-13 Thread Bikas Saha (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-2681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13186074#comment-13186074
 ] 

Bikas Saha commented on HDFS-2681:
--

Some more comments from chat session
# Change the method name to init(). Annotate it @Before. It will be 
automatically called before tests.
# Use @Expected for tests that expect exception
Done
# Add class level javadoc.
This is already in the second patch. If by this you mean comments before the 
class declaration.
# #Init need not catch IOException. Just throw it. The test will fail.
I know it wont throw the exception in the test. But it has to be handled to 
keep the compiler happy. So I handled it locally there instead of adding 
throws IOException in every method of the test
# You can reduce several lines of code by using a static byte[] DATA;
done
# can you add test where jointElection() is called twice and the second call is 
NO-OP
# Many times where processResult is called back to back can be in for loop
this helps me walk the scenarios better than in a loop
# Why should 4 errors of connection loss result in fatalError?
Because the elector has tried its best to connect to Zookeeper and failed. We 
can revisit this based observed failures at a later time.
# testStatNodeError already covers some part of 
testCreateNodeResultRetryBecomeActive
yes. thats because it is trying to walk through a logical scenario. so I let it 
be.
# Instead of catching InterruptedException, you can just throw it
same code cleanliness as above. I know this exception will not get thrown in 
the test. so want to make local changes to keep the compiler happy.

Please use System.arraycopy() instead of byte[] clone.
done.
Split process into two different methods processZkEvent and processZnodeEvent?
The function is still small enough to let it be. Will do this later when more 
logic might get added if we do group participation. At that point 
processZnodeEvent itself will need division into lock znode and parent znode.

can you add test where jointElection() is called twice and the second call is 
NO-OP
it was there in test processResult callback but got changed to enterNeutralMode 
when I changed that test. now I enhanced testCreateNodeResultBecomeActive() to 
check that there is no double master call and added another test to check that 
there is no double slave call for expected scenarios. now all 3 states are 
covered.

Will upload the patch with all these changes.
Thanks


 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, 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