[jira] [Commented] (RATIS-377) Tolerate partially written log header

2018-10-29 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/RATIS-377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16668147#comment-16668147
 ] 

Tsz Wo Nicholas Sze commented on RATIS-377:
---

> It seems if HEADER_BYTES_CLONE is modified. ...

How could this be possible?  By modifying the source code?

> Can we consider using something such as 
> https://static.javadoc.io/org.apache.fluo/fluo-api/1.2.0/org/apache/fluo/api/data/Bytes.html
>  ?

Thanks for the pointer.  Bytes look likes a nice class but it does not make 
sense to add Flue as a dependency of Ratis just for it.

> Tolerate partially written log header
> -
>
> Key: RATIS-377
> URL: https://issues.apache.org/jira/browse/RATIS-377
> Project: Ratis
>  Issue Type: Bug
>Affects Versions: 0.3.0
>Reporter: Nilotpal Nandi
>Assignee: Tsz Wo Nicholas Sze
>Priority: Blocker
> Fix For: 0.3.0
>
> Attachments: r377_20181028c.patch
>
>
> steps taken :
> --
>  # wrote 5GB files through ozonefs
>  # stopped datanodes, scm , om.
>  # started all services.
>  # Tried to read the file.
> One of the datanodes failed to start. Throwing 
> "java.lang.IllegalStateException: Corrupted log header" 
>  
> {noformat}
> 2018-10-26 10:26:01,317 ERROR org.apache.ratis.server.storage.LogInputStream: 
> caught exception initializing log_inprogress_293
> java.lang.IllegalStateException: Corrupted log header: ^@^@^@^@^@^@^@^@
>  at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
>  at 
> org.apache.ratis.server.storage.LogInputStream.init(LogInputStream.java:93)
>  at 
> org.apache.ratis.server.storage.LogInputStream.nextEntry(LogInputStream.java:120)
>  at 
> org.apache.ratis.server.storage.LogSegment.readSegmentFile(LogSegment.java:111)
>  at 
> org.apache.ratis.server.storage.LogSegment.loadSegment(LogSegment.java:133)
>  at 
> org.apache.ratis.server.storage.RaftLogCache.loadSegment(RaftLogCache.java:110)
>  at 
> org.apache.ratis.server.storage.SegmentedRaftLog.loadLogSegments(SegmentedRaftLog.java:151)
>  at 
> org.apache.ratis.server.storage.SegmentedRaftLog.open(SegmentedRaftLog.java:120)
>  at org.apache.ratis.server.impl.ServerState.initLog(ServerState.java:191)
>  at org.apache.ratis.server.impl.ServerState.(ServerState.java:114)
>  at 
> org.apache.ratis.server.impl.RaftServerImpl.(RaftServerImpl.java:106)
>  at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:196)
>  at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
>  at 
> java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1582)
>  at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
>  at 
> java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
>  at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
>  at 
> java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157)
> 2018-10-26 10:26:03,671 INFO 
> org.apache.hadoop.ozone.web.netty.ObjectStoreRestHttpServer: Listening HDDS 
> REST traffic on /0.0.0.0:9880
> 2018-10-26 10:26:03,672 INFO org.apache.hadoop.ozone.HddsDatanodeService: 
> Started plug-in org.apache.hadoop.ozone.web.OzoneHddsDatanodeService@1e411d81
> 2018-10-26 10:26:03,676 INFO 
> org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer: Attempting to 
> start container services.
> 2018-10-26 10:26:03,676 INFO 
> org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis:
>  Starting XceiverServerRatis 0d7f5327-df16-40fe-ac88-7ed06e76a20f at port 9858
> 2018-10-26 10:26:03,702 ERROR 
> org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine: 
> Unable to start the DatanodeState Machine
> java.io.IOException: java.lang.IllegalStateException: Corrupted log header: 
> ^@^@^@^@^@^@^@^@
>  at org.apache.ratis.util.IOUtils.asIOException(IOUtils.java:51)
>  at 
> org.apache.ratis.server.storage.LogInputStream.nextEntry(LogInputStream.java:123)
>  at 
> org.apache.ratis.server.storage.LogSegment.readSegmentFile(LogSegment.java:111)
>  at 
> org.apache.ratis.server.storage.LogSegment.loadSegment(LogSegment.java:133)
>  at 
> org.apache.ratis.server.storage.RaftLogCache.loadSegment(RaftLogCache.java:110)
>  at 
> org.apache.ratis.server.storage.SegmentedRaftLog.loadLogSegments(SegmentedRaftLog.java:151)
>  at 
> org.apache.ratis.server.storage.SegmentedRaftLog.open(SegmentedRaftLog.java:120)
>  at org.apache.ratis.server.impl.ServerState.initLog(ServerState.java:191)
>  at org.apache.ratis.server.impl.ServerState.(ServerState.java:114)
>  at 
> org.apache.ratis.server.impl.RaftServerImpl.(RaftServerImpl.java:106)
>  at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:196)
>  at 
> 

[jira] [Commented] (RATIS-379) Allow writing state machine data to be sync'ed with writing raft log

2018-10-29 Thread Jitendra Nath Pandey (JIRA)


[ 
https://issues.apache.org/jira/browse/RATIS-379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16668139#comment-16668139
 ] 

Jitendra Nath Pandey commented on RATIS-379:


I see, thanks for clarification.

+1 for the latest patch.

> Allow writing state machine data to be sync'ed with writing raft log
> 
>
> Key: RATIS-379
> URL: https://issues.apache.org/jira/browse/RATIS-379
> Project: Ratis
>  Issue Type: Improvement
>  Components: server
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: r379_20181030.patch
>
>
> Currently, writing state machine data and writing raft log are both 
> asynchronous so that they can be completed in any order.  In some 
> applications such as Ozone, it is hard to handle the case that writing raft 
> log has succeeded but state machine data has failed.
> In this JIRA, we provide an option to allow writing state machine data to be 
> synchronized with writing raft log.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (RATIS-377) Tolerate partially written log header

2018-10-29 Thread Ted Yu (JIRA)


[ 
https://issues.apache.org/jira/browse/RATIS-377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16667813#comment-16667813
 ] 

Ted Yu commented on RATIS-377:
--

In LogSegment#loadSegment :
{code}
if (entryCount == 0) {
  // The segment does not have any entries, delete the file.
  FileUtils.deleteFile(file);
{code}
Call to deleteFile may throw IOE.
However, I think for loadSegment, such IOE can be caught and logged without 
surfacing to the caller.

> Tolerate partially written log header
> -
>
> Key: RATIS-377
> URL: https://issues.apache.org/jira/browse/RATIS-377
> Project: Ratis
>  Issue Type: Bug
>Affects Versions: 0.3.0
>Reporter: Nilotpal Nandi
>Assignee: Tsz Wo Nicholas Sze
>Priority: Blocker
> Fix For: 0.3.0
>
> Attachments: r377_20181028c.patch
>
>
> steps taken :
> --
>  # wrote 5GB files through ozonefs
>  # stopped datanodes, scm , om.
>  # started all services.
>  # Tried to read the file.
> One of the datanodes failed to start. Throwing 
> "java.lang.IllegalStateException: Corrupted log header" 
>  
> {noformat}
> 2018-10-26 10:26:01,317 ERROR org.apache.ratis.server.storage.LogInputStream: 
> caught exception initializing log_inprogress_293
> java.lang.IllegalStateException: Corrupted log header: ^@^@^@^@^@^@^@^@
>  at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
>  at 
> org.apache.ratis.server.storage.LogInputStream.init(LogInputStream.java:93)
>  at 
> org.apache.ratis.server.storage.LogInputStream.nextEntry(LogInputStream.java:120)
>  at 
> org.apache.ratis.server.storage.LogSegment.readSegmentFile(LogSegment.java:111)
>  at 
> org.apache.ratis.server.storage.LogSegment.loadSegment(LogSegment.java:133)
>  at 
> org.apache.ratis.server.storage.RaftLogCache.loadSegment(RaftLogCache.java:110)
>  at 
> org.apache.ratis.server.storage.SegmentedRaftLog.loadLogSegments(SegmentedRaftLog.java:151)
>  at 
> org.apache.ratis.server.storage.SegmentedRaftLog.open(SegmentedRaftLog.java:120)
>  at org.apache.ratis.server.impl.ServerState.initLog(ServerState.java:191)
>  at org.apache.ratis.server.impl.ServerState.(ServerState.java:114)
>  at 
> org.apache.ratis.server.impl.RaftServerImpl.(RaftServerImpl.java:106)
>  at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:196)
>  at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
>  at 
> java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1582)
>  at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
>  at 
> java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
>  at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
>  at 
> java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157)
> 2018-10-26 10:26:03,671 INFO 
> org.apache.hadoop.ozone.web.netty.ObjectStoreRestHttpServer: Listening HDDS 
> REST traffic on /0.0.0.0:9880
> 2018-10-26 10:26:03,672 INFO org.apache.hadoop.ozone.HddsDatanodeService: 
> Started plug-in org.apache.hadoop.ozone.web.OzoneHddsDatanodeService@1e411d81
> 2018-10-26 10:26:03,676 INFO 
> org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer: Attempting to 
> start container services.
> 2018-10-26 10:26:03,676 INFO 
> org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis:
>  Starting XceiverServerRatis 0d7f5327-df16-40fe-ac88-7ed06e76a20f at port 9858
> 2018-10-26 10:26:03,702 ERROR 
> org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine: 
> Unable to start the DatanodeState Machine
> java.io.IOException: java.lang.IllegalStateException: Corrupted log header: 
> ^@^@^@^@^@^@^@^@
>  at org.apache.ratis.util.IOUtils.asIOException(IOUtils.java:51)
>  at 
> org.apache.ratis.server.storage.LogInputStream.nextEntry(LogInputStream.java:123)
>  at 
> org.apache.ratis.server.storage.LogSegment.readSegmentFile(LogSegment.java:111)
>  at 
> org.apache.ratis.server.storage.LogSegment.loadSegment(LogSegment.java:133)
>  at 
> org.apache.ratis.server.storage.RaftLogCache.loadSegment(RaftLogCache.java:110)
>  at 
> org.apache.ratis.server.storage.SegmentedRaftLog.loadLogSegments(SegmentedRaftLog.java:151)
>  at 
> org.apache.ratis.server.storage.SegmentedRaftLog.open(SegmentedRaftLog.java:120)
>  at org.apache.ratis.server.impl.ServerState.initLog(ServerState.java:191)
>  at org.apache.ratis.server.impl.ServerState.(ServerState.java:114)
>  at 
> org.apache.ratis.server.impl.RaftServerImpl.(RaftServerImpl.java:106)
>  at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:196)
>  at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
>  at 
> 

[jira] [Created] (RATIS-380) RaftGroup#hashCode should take peers into account

2018-10-29 Thread Ted Yu (JIRA)
Ted Yu created RATIS-380:


 Summary: RaftGroup#hashCode should take peers into account
 Key: RATIS-380
 URL: https://issues.apache.org/jira/browse/RATIS-380
 Project: Ratis
  Issue Type: Improvement
Reporter: Ted Yu


Currently RaftGroup#hashCode produces hash code only based on groupId.

This is unsymmetrical with {{equals}} where peers is also considered.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (RATIS-379) Allow writing state machine data to be sync'ed with writing raft log

2018-10-29 Thread Jitendra Nath Pandey (JIRA)


[ 
https://issues.apache.org/jira/browse/RATIS-379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16667590#comment-16667590
 ] 

Jitendra Nath Pandey commented on RATIS-379:


Call to {{getFromFuture}} is no longer needed in flushWrites, because it is 
already called in {{execute}} however it should return immediately.

> Allow writing state machine data to be sync'ed with writing raft log
> 
>
> Key: RATIS-379
> URL: https://issues.apache.org/jira/browse/RATIS-379
> Project: Ratis
>  Issue Type: Improvement
>  Components: server
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: r379_20181030.patch
>
>
> Currently, writing state machine data and writing raft log are both 
> asynchronous so that they can be completed in any order.  In some 
> applications such as Ozone, it is hard to handle the case that writing raft 
> log has succeeded but state machine data has failed.
> In this JIRA, we provide an option to allow writing state machine data to be 
> synchronized with writing raft log.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (RATIS-379) Allow writing state machine data to be sync'ed with writing raft log

2018-10-29 Thread Hadoop QA (JIRA)


[ 
https://issues.apache.org/jira/browse/RATIS-379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16667547#comment-16667547
 ] 

Hadoop QA commented on RATIS-379:
-

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue}  4m  
3s{color} | {color:blue} Docker mode activated. {color} |
|| || || || {color:brown} Prechecks {color} ||
| {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue}  0m  
0s{color} | {color:blue} Findbugs executables are not available. {color} |
| {color:green}+1{color} | {color:green} @author {color} | {color:green}  0m  
0s{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:green}+1{color} | {color:green} test4tests {color} | {color:green}  0m 
 0s{color} | {color:green} The patch appears to include 3 new or modified test 
files. {color} |
|| || || || {color:brown} master Compile Tests {color} ||
| {color:blue}0{color} | {color:blue} mvndep {color} | {color:blue}  1m 
23s{color} | {color:blue} Maven dependency ordering for branch {color} |
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green}  1m 
52s{color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  0m 
43s{color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
26s{color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
37s{color} | {color:green} master passed {color} |
|| || || || {color:brown} Patch Compile Tests {color} ||
| {color:blue}0{color} | {color:blue} mvndep {color} | {color:blue}  0m  
5s{color} | {color:blue} Maven dependency ordering for patch {color} |
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green}  0m 
50s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  0m 
44s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green}  0m 
44s{color} | {color:green} the patch passed {color} |
| {color:orange}-0{color} | {color:orange} checkstyle {color} | {color:orange}  
0m 22s{color} | {color:orange} root: The patch generated 61 new + 569 unchanged 
- 19 fixed = 630 total (was 588) {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green}  0m 
 0s{color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
31s{color} | {color:green} the patch passed {color} |
|| || || || {color:brown} Other Tests {color} ||
| {color:red}-1{color} | {color:red} unit {color} | {color:red}  5m  5s{color} 
| {color:red} root in the patch failed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green}  0m 
14s{color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black} 17m 29s{color} | 
{color:black} {color} |
\\
\\
|| Reason || Tests ||
| Failed junit tests | 
ratis.server.simulation.TestServerRestartWithSimulatedRpc |
|   | ratis.server.simulation.TestRaftWithSimulatedRpc |
|   | ratis.server.simulation.TestRaftStateMachineExceptionWithSimulatedRpc |
\\
\\
|| Subsystem || Report/Notes ||
| Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/ratis:date2018-10-29 
|
| JIRA Issue | RATIS-379 |
| JIRA Patch URL | 
https://issues.apache.org/jira/secure/attachment/12946073/r379_20181030.patch |
| Optional Tests |  asflicense  javac  javadoc  unit  findbugs  checkstyle  
compile  |
| uname | Linux 90b577817c3c 3.13.0-143-generic #192-Ubuntu SMP Tue Feb 27 
10:45:36 UTC 2018 x86_64 x86_64 x86_64 GNU/Linux |
| Build tool | maven |
| Personality | 
/home/jenkins/jenkins-slave/workspace/PreCommit-RATIS-Build/yetus-personality.sh
 |
| git revision | master / 4d723a2 |
| Default Java | 1.8.0_181 |
| checkstyle | 
https://builds.apache.org/job/PreCommit-RATIS-Build/475/artifact/out/diff-checkstyle-root.txt
 |
| unit | 
https://builds.apache.org/job/PreCommit-RATIS-Build/475/artifact/out/patch-unit-root.txt
 |
|  Test Results | 
https://builds.apache.org/job/PreCommit-RATIS-Build/475/testReport/ |
| modules | C: ratis-common ratis-server U: . |
| Console output | 
https://builds.apache.org/job/PreCommit-RATIS-Build/475/console |
| Powered by | Apache Yetus 0.5.0   http://yetus.apache.org |


This message was automatically generated.



> Allow writing state machine data to be sync'ed with writing raft log
> 
>
> Key: RATIS-379
> URL: https://issues.apache.org/jira/browse/RATIS-379
> Project: Ratis
>  Issue 

[jira] [Created] (RATIS-379) Allow writing state machine data to be sync'ed with writing raft log

2018-10-29 Thread Tsz Wo Nicholas Sze (JIRA)
Tsz Wo Nicholas Sze created RATIS-379:
-

 Summary: Allow writing state machine data to be sync'ed with 
writing raft log
 Key: RATIS-379
 URL: https://issues.apache.org/jira/browse/RATIS-379
 Project: Ratis
  Issue Type: Improvement
  Components: server
Reporter: Tsz Wo Nicholas Sze
Assignee: Tsz Wo Nicholas Sze


Currently, writing state machine data and writing raft log are both 
asynchronous so that they can be completed in any order.  In some applications 
such as Ozone, it is hard to handle the case that writing raft log has 
succeeded but state machine data has failed.

In this JIRA, we provide an option to allow writing state machine data to be 
synchronized with writing raft log.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)