[jira] [Commented] (RATIS-160) Retry cache should handle leader change after log commit

2018-01-16 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-160:
-

The patch looks good to me. The only minor comment is that it may be better to 
add the new replacing-the-failed-entry code in {{replyPendingRequest}} after 
calling {{getOrCreateEntry}}, instead of directly into {{getOrCreateEntry}}. We 
may want to keep {{getOrCreateEntry}} as a simple method handling the data 
structure only. 
{code}
166   entry = cache.get(key, () -> new CacheEntry(key));
167   if (entry.isFailed()) {
168 refreshEntry(new CacheEntry(key));
169   }
{code}

> Retry cache should handle leader change after log commit
> 
>
> Key: RATIS-160
> URL: https://issues.apache.org/jira/browse/RATIS-160
> Project: Ratis
>  Issue Type: Bug
>Reporter: Lokesh Jain
>Assignee: Lokesh Jain
>Priority: Major
> Attachments: RATIS-160.001.patch, RATIS-160.002.patch
>
>
> This jira is in relation to the below exception seen in the logs. 
> {code:java}
> java.lang.IllegalStateException: retry cache entry should be pending: 
> client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> at java.lang.Thread.run(Thread.java:748)
> Exception in thread "StateMachineUpdater-s3" 
> org.apache.ratis.util.ExitUtils$ExitException: StateMachineUpdater-s3: the 
> StateMachineUpdater hits Throwable
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:94)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:175)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: retry cache entry should be 
> pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> ... 1 more
> {code}
> This occurs when leader commits a log entry but is not able to send a reply 
> to the client before leader is changed. When the new leader gets the request 
> it sends the append entry request to the followers whose cache already has 
> the  corresponding entry leading to the above exception.



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


[jira] [Commented] (RATIS-140) Raft client should reuse the gRPC stream for all async calls

2017-12-27 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-140:
-

Looks like the failed test is unrelated. Could you please confirm it? Other 
than this the patch looks good to me. +1

> Raft client should reuse the gRPC stream for all async calls
> 
>
> Key: RATIS-140
> URL: https://issues.apache.org/jira/browse/RATIS-140
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r140_20171123.patch, r140_20171124.patch, 
> r140_20171125.patch, r140_20171126.patch, r140_20171126b.patch, 
> r140_20171130.patch, r140_20171203.patch, r140_20171204.patch, 
> r140_20171206.patch, r140_20171210.patch, r140_20171219.patch
>
>
> Async client is being added in RATIS-113.  However, we found that the server 
> side (RaftClientProtocolService) may see out-of-order grpc messages even if 
> all messages are sent by the same client.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-140) Raft client should reuse the gRPC stream for all async calls

2017-12-18 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-140:
-

Some comments so far (after discussion with Nicholas offline):
# Maybe we should use composition instead of inheritance for Client and 
RequestMap? Letting Client extends RequestMap may not be that direct.
# Currently the size limitation of the total number of outstanding requests is 
defined in RaftClientImpl. I think to have the size limitation inside of 
SlidingWindow may be better.
# We may still need a better abstraction for SlidingWindow. Maybe we should 
make it a pure data structure and avoid actions like "submissionMethod" inside 
of it as a member field. This abstraction later can also be adopted by 
AppendStreamer and GrpcLogAppender.
# Currently for gRPC module, this patch actually cuts the underlying grpc 
streamer into small pieces (each onNext call), and then builds a consecutive 
async RPC call model on top. To me this may not be clean. We may want to 
revisit our API choice later (sendAsync v.s. OutputStream) in a separate jira.

I think we can fix some minor issues and commit the patch in this jira. Then we 
can create follow-on jiras to discuss the above #3 and #4 comments.

> Raft client should reuse the gRPC stream for all async calls
> 
>
> Key: RATIS-140
> URL: https://issues.apache.org/jira/browse/RATIS-140
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r140_20171123.patch, r140_20171124.patch, 
> r140_20171125.patch, r140_20171126.patch, r140_20171126b.patch, 
> r140_20171130.patch, r140_20171203.patch, r140_20171204.patch, 
> r140_20171206.patch, r140_20171210.patch
>
>
> Async client is being added in RATIS-113.  However, we found that the server 
> side (RaftClientProtocolService) may see out-of-order grpc messages even if 
> all messages are sent by the same client.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-140) Raft client should reuse the gRPC stream for all async calls

2017-12-12 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-140:
-

Thanks for updating the patch, Nicholas! I will post comments these days.

> Raft client should reuse the gRPC stream for all async calls
> 
>
> Key: RATIS-140
> URL: https://issues.apache.org/jira/browse/RATIS-140
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r140_20171123.patch, r140_20171124.patch, 
> r140_20171125.patch, r140_20171126.patch, r140_20171126b.patch, 
> r140_20171130.patch, r140_20171203.patch, r140_20171204.patch, 
> r140_20171206.patch, r140_20171210.patch
>
>
> Async client is being added in RATIS-113.  However, we found that the server 
> side (RaftClientProtocolService) may see out-of-order grpc messages even if 
> all messages are sent by the same client.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-160) Retry cache should handle leader change after log commit

2017-12-10 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-160:
-

bq. Actually the bug occurs when new leader creates a new log entry suppose e2 
for the entry which is already in log e1 and then tries to commit it. It is 
able to create a new log entry because the retry cache entry for the entry e1 
is failed. 

Yes, I understand how we hit the exception But still, the real cause of the 
bug is, when s1 becomes the leader again (in the step 5 of you described 
scenario), it should correctly replace a failed retry cache entry when applying 
log entries to its state machine. If we fix that part, when the client's retry 
request comes, the retry cache works as expected.

Let me try to explain why your current fix is not good from another 
perspective. In your patch, the main usage of the new "entryInLog" is:
{code}
202 } else if (!cacheEntry.isDone() || !cacheEntry.isFailed() || 
cacheEntry.entryInLog){
203   // the previous attempt is either pending or successful
204   return new CacheQueryResult(cacheEntry, true);
205 }
{code}
This means the code also returns a cache query result if we find that the entry 
is 1) done, and 2) failed, and 3) has entryInLog set to true. This is wrong. At 
this scenario, we should NOT return the query result. Instead we should replace 
the original retry cache entry with a new pending one.

This is a tricky issue. Will be happy to discuss it offline through google 
hangout.

> Retry cache should handle leader change after log commit
> 
>
> Key: RATIS-160
> URL: https://issues.apache.org/jira/browse/RATIS-160
> Project: Ratis
>  Issue Type: Bug
>Reporter: Lokesh Jain
>Assignee: Lokesh Jain
> Attachments: RATIS-160.001.patch
>
>
> This jira is in relation to the below exception seen in the logs. 
> {code:java}
> java.lang.IllegalStateException: retry cache entry should be pending: 
> client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> at java.lang.Thread.run(Thread.java:748)
> Exception in thread "StateMachineUpdater-s3" 
> org.apache.ratis.util.ExitUtils$ExitException: StateMachineUpdater-s3: the 
> StateMachineUpdater hits Throwable
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:94)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:175)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: retry cache entry should be 
> pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> ... 1 more
> {code}
> This occurs when leader commits a log entry but is not able to send a reply 
> to the client before leader is changed. When the new leader gets the request 
> it sends the append entry request to the followers whose cache already has 
> the  corresponding entry leading to the above exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-160) Retry cache should handle leader change after log commit

2017-12-10 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-160:
-

bq. The exception is raised by follower when they see a completed cache entry 
when trying to apply log entry to the state machine. I can remove the 
Preconditions check but it would lead to same request being applied twice to 
the state machine.

Here my proposed fix is not to remove the preconditions, but in 
{{replyPendingRequest}}, after calling {{retryCache.getOrCreateEntry}}, add an 
extra check whether cacheEntry has been marked as "failed". If yes, we replace 
the entry with a pending entry.

The bug happens actually in the new leader, while it is committing log entries 
in the previous term. Before, we thought during this stage the leader could 
only see null or  a pending retry cache entry in the retry cache. Now we find 
the exception here.

bq. Since the log entries are applied serially to the state machine, it would 
ultimately be committed by the followers on append entry request of leader.
No, it's incorrect. The leader needs to first commit the log entry to the raft 
group (i.e., to persist the log entry to the quorum), then it applies the log 
entry to the state machine. When you mark "entryInLog" to true in the current 
patch, it is too early to know whether the log entry would be committed 
successfully.

> Retry cache should handle leader change after log commit
> 
>
> Key: RATIS-160
> URL: https://issues.apache.org/jira/browse/RATIS-160
> Project: Ratis
>  Issue Type: Bug
>Reporter: Lokesh Jain
>Assignee: Lokesh Jain
> Attachments: RATIS-160.001.patch
>
>
> This jira is in relation to the below exception seen in the logs. 
> {code:java}
> java.lang.IllegalStateException: retry cache entry should be pending: 
> client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> at java.lang.Thread.run(Thread.java:748)
> Exception in thread "StateMachineUpdater-s3" 
> org.apache.ratis.util.ExitUtils$ExitException: StateMachineUpdater-s3: the 
> StateMachineUpdater hits Throwable
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:94)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:175)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: retry cache entry should be 
> pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> ... 1 more
> {code}
> This occurs when leader commits a log entry but is not able to send a reply 
> to the client before leader is changed. When the new leader gets the request 
> it sends the append entry request to the followers whose cache already has 
> the  corresponding entry leading to the above exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (RATIS-160) Retry cache should handle leader change after log commit

2017-12-10 Thread Jing Zhao (JIRA)

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

Jing Zhao edited comment on RATIS-160 at 12/11/17 3:11 AM:
---

The reason I do not like adding "entryInLog" is that its semantic meaning is 
unclear. Actually in the current patch this boolean field only captures whether 
the retry cache entry is in an in-memory log segment. It cannot tell us whether 
the entry has be persisted in the disk, or has been committed to the raft 
group. Note the corresponding log entry can still be omitted or even truncated. 
Therefore even if we record "entryInLog" as true, we actually cannot guarantee 
anything.

Do you think the proposed fix in my previous comment can fix the issue? If yes 
that should be a simple fix. In the meanwhile, we should add a unit test to 
reproduce the scenario you observed. But that may be complicated so we can do 
it in a separate jira.


was (Author: jingzhao):
The reason I do not like adding "entryInLog" is that its semantic meaning is 
unclear. Actually the current patch only captures whether the retry cache entry 
is in an in-memory log segment. It cannot tell us whether the entry has be 
persisted in the disk, or has been committed to the raft group. Note the 
corresponding log entry can still be omitted or even truncated. Therefore even 
if we record "entryInLog" as true, we actually cannot guarantee anything.

Do you think the proposed fix in my previous comment can fix the issue? If yes 
that should be a simple fix. In the meanwhile, we should add a unit test to 
reproduce the scenario you observed. But that may be complicated so we can do 
it in a separate jira.

> Retry cache should handle leader change after log commit
> 
>
> Key: RATIS-160
> URL: https://issues.apache.org/jira/browse/RATIS-160
> Project: Ratis
>  Issue Type: Bug
>Reporter: Lokesh Jain
>Assignee: Lokesh Jain
> Attachments: RATIS-160.001.patch
>
>
> This jira is in relation to the below exception seen in the logs. 
> {code:java}
> java.lang.IllegalStateException: retry cache entry should be pending: 
> client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> at java.lang.Thread.run(Thread.java:748)
> Exception in thread "StateMachineUpdater-s3" 
> org.apache.ratis.util.ExitUtils$ExitException: StateMachineUpdater-s3: the 
> StateMachineUpdater hits Throwable
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:94)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:175)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: retry cache entry should be 
> pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> ... 1 more
> {code}
> This occurs when leader commits a log entry but is not able to send a reply 
> to the client before leader is changed. When the new leader gets the request 
> it sends the append entry request to the followers whose cache already has 
> the  corresponding entry leading to the above exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-160) Retry cache should handle leader change after log commit

2017-12-09 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-160:
-

Thanks a lot Lokesh! I've downloaded the log and will dig into the issue this 
weekend.

> Retry cache should handle leader change after log commit
> 
>
> Key: RATIS-160
> URL: https://issues.apache.org/jira/browse/RATIS-160
> Project: Ratis
>  Issue Type: Bug
>Reporter: Lokesh Jain
>Assignee: Lokesh Jain
> Attachments: RATIS-160.001.patch
>
>
> This jira is in relation to the below exception seen in the logs. 
> {code:java}
> java.lang.IllegalStateException: retry cache entry should be pending: 
> client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> at java.lang.Thread.run(Thread.java:748)
> Exception in thread "StateMachineUpdater-s3" 
> org.apache.ratis.util.ExitUtils$ExitException: StateMachineUpdater-s3: the 
> StateMachineUpdater hits Throwable
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:94)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:175)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: retry cache entry should be 
> pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> ... 1 more
> {code}
> This occurs when leader commits a log entry but is not able to send a reply 
> to the client before leader is changed. When the new leader gets the request 
> it sends the append entry request to the followers whose cache already has 
> the  corresponding entry leading to the above exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-160) Retry cache should handle leader change after log commit

2017-12-07 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-160:
-

Thanks again for reporting the issue, Lokesh.
bq. This occurs when leader commits a log entry but is not able to send a reply 
to the client before leader is changed. When the new leader gets the request it 
sends the append entry request to the followers whose cache already has the 
corresponding entry leading to the above exception.
First if a log entry has been committed, the new leader must have that log 
entry, and the original cache entry should be inside of the new leader's retry 
cache. Besides, after RATIS-18 a new leader is allowed to serve client requests 
only after it applies the placeholder entry in the new term, which means the 
entries in the previous term can be reflected in the retry cache.

I checked the current code but so far I'm not sure if the scenario created by 
{{testChangeLeaderBeforeLogCommit}} can happen in a real scenario. 
{{RetryCache#failWithException}} and {{RetryCache#failWithReply}} are called 
only when the leader receives a client request, but before these two methods 
get called the leader should have already checked the retry cache.

[~ljain], could you please share more details about how we can reproduce the 
issue in a raft group? If possible, could you please share some log or detailed 
steps for reproducing the issue?



> Retry cache should handle leader change after log commit
> 
>
> Key: RATIS-160
> URL: https://issues.apache.org/jira/browse/RATIS-160
> Project: Ratis
>  Issue Type: Bug
>Reporter: Lokesh Jain
>Assignee: Lokesh Jain
> Attachments: RATIS-160.001.patch
>
>
> This jira is in relation to the below exception seen in the logs. 
> {code:java}
> java.lang.IllegalStateException: retry cache entry should be pending: 
> client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> at java.lang.Thread.run(Thread.java:748)
> Exception in thread "StateMachineUpdater-s3" 
> org.apache.ratis.util.ExitUtils$ExitException: StateMachineUpdater-s3: the 
> StateMachineUpdater hits Throwable
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:94)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:175)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: retry cache entry should be 
> pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> ... 1 more
> {code}
> This occurs when leader commits a log entry but is not able to send a reply 
> to the client before leader is changed. When the new leader gets the request 
> it sends the append entry request to the followers whose cache already has 
> the  corresponding entry leading to the above exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-160) Retry cache should handle leader change after log commit

2017-12-06 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-160:
-

Thanks for working on this, [~ljain]. I will review your patch today/tomorrow.

> Retry cache should handle leader change after log commit
> 
>
> Key: RATIS-160
> URL: https://issues.apache.org/jira/browse/RATIS-160
> Project: Ratis
>  Issue Type: Bug
>Reporter: Lokesh Jain
>Assignee: Lokesh Jain
> Attachments: RATIS-160.001.patch
>
>
> This jira is in relation to the below exception seen in the logs. 
> {code:java}
> java.lang.IllegalStateException: retry cache entry should be pending: 
> client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> at java.lang.Thread.run(Thread.java:748)
> Exception in thread "StateMachineUpdater-s3" 
> org.apache.ratis.util.ExitUtils$ExitException: StateMachineUpdater-s3: the 
> StateMachineUpdater hits Throwable
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:94)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:175)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: retry cache entry should be 
> pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
> at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
> at 
> org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
> at 
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
> ... 1 more
> {code}
> This occurs when leader commits a log entry but is not able to send a reply 
> to the client before leader is changed. When the new leader gets the request 
> it sends the append entry request to the followers whose cache already has 
> the  corresponding entry leading to the above exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-140) Server may see out-of-order gRPC messages sent from the same client

2017-12-04 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-140:
-

Thanks a lot, Nicholas! For gRpc, my feeling is we have duplicated 
functionalities on the client side. We already call GRpc's async/streaming API 
in {{RaftClientProtocolClient#append}} and {{RaftClientProtocolProxy#onNext}}. 
In {{AppendStreamer}} we utilize this async/streaming call to achieve an 
efficient transferring implementation (as this jira aims to do). And before 
RATIS-113, {{GrpcClientRpc#sendRequest}} called the streaming API only once and 
wrapped this async call into a synchronous call. 

I think the change by RATIS-113 on the gRpc part is unnecessary and incorrect. 
It wraps a single {{onNext}} call of the gRpc streamer into 
{{sendRequestAsync}}, while by gRpc's design the streamer's {{onNext}} API 
should be called consecutively to achieve async and streaming (as we do in 
{{AppendStreamer}}). This is the reason we see out-of-order gRPC msgs from the 
same client after RATIS-113.

For this jira, at least for gRPC part, I do not think we should continue 
building a new OutputStream on top of {{sendRequestAsync}}. But the new 
{{SlidingWindow}} is a great abstraction and we can use it to consolidate the 
streaming implementation of AppendStreamer and GRpcLogAppender.

> Server may see out-of-order gRPC messages sent from the same client
> ---
>
> Key: RATIS-140
> URL: https://issues.apache.org/jira/browse/RATIS-140
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r140_20171123.patch, r140_20171124.patch, 
> r140_20171125.patch, r140_20171126.patch, r140_20171126b.patch, 
> r140_20171130.patch, r140_20171203.patch, r140_20171204.patch
>
>
> Async client is being added in RATIS-113.  However, we found that the server 
> side (RaftClientProtocolService) may see out-of-order grpc messages even if 
> all messages are sent by the same client.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-6) Project logo

2017-11-20 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-6:
---

+1 on logo-finalist.png

> Project logo
> 
>
> Key: RATIS-6
> URL: https://issues.apache.org/jira/browse/RATIS-6
> Project: Ratis
>  Issue Type: Task
>Reporter: Enis Soztutar
>Assignee: Will Xu
> Attachments: Artboard 2.png, Ratis-Logo.png, Ratis.png, 
> logo-finalist.png
>
>




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Assigned] (RATIS-149) TestRaftStream.testSimpleWrite may fail

2017-11-20 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-149?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao reassigned RATIS-149:
---

Assignee: Jing Zhao

> TestRaftStream.testSimpleWrite may fail
> ---
>
> Key: RATIS-149
> URL: https://issues.apache.org/jira/browse/RATIS-149
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Jing Zhao
>
> Two different failure cases:
> - {code}
> java.lang.AssertionError: expected:<500> but was:<350>
>   at 
> org.apache.ratis.grpc.TestRaftStream.checkLog(TestRaftStream.java:106)
>   at 
> org.apache.ratis.grpc.TestRaftStream.testSimpleWrite(TestRaftStream.java:100)
> {code}
> - {code}
> org.junit.internal.ArrayComparisonFailure: arrays first differed at element 
> [0]; expected:<63> but was:<-81>
>   at 
> org.apache.ratis.grpc.TestRaftStream.checkLog(TestRaftStream.java:114)
>   at 
> org.apache.ratis.grpc.TestRaftStream.testSimpleWrite(TestRaftStream.java:100)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Resolved] (RATIS-142) Test ArithmeticStateMachine with the Gauss–Legendre algorithm

2017-11-17 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-142?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-142.
-
   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

I've committed the patch. Thanks Nicholas!

> Test ArithmeticStateMachine with the Gauss–Legendre algorithm
> -
>
> Key: RATIS-142
> URL: https://issues.apache.org/jira/browse/RATIS-142
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Minor
> Fix For: 0.2.0-alpha
>
> Attachments: r142_20171114.patch, r142_20171116.patch
>
>
> The Gauss–Legendre algorithm, a.k.a. the arithmetic–geometric mean method, is 
> a fast algorithm to compute pi; see 
> https://en.wikipedia.org/wiki/Gauss%E2%80%93Legendre_algorithm
> We use it to test the ArithmeticStateMachine example.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-141) In RaftClientProtocolService, the assumption of consecutive callId is invalid

2017-11-13 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-141:
-

Yeah, sounds good to me. In the meanwhile, let me check why we made this 
assumption in the very beginning.

> In RaftClientProtocolService, the assumption of consecutive callId is invalid
> -
>
> Key: RATIS-141
> URL: https://issues.apache.org/jira/browse/RATIS-141
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>
> {code}
> //RaftClientProtocolService.AppendRequestStreamObserver.onNext(..)
>   // we assume the callId is consecutive for a stream RPC call
>   final PendingAppend pendingForReply = pendingList.get(
>   (int) (replySeq - headSeqNum));
> {code}
> Call id is used for different kinds of calls (e.g. getInfo) so that it may 
> not be consecutive.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-133) Raft gRPC client should check proto size before sending a message

2017-11-10 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-133:
-

[~msingh], I see your point. Thanks for the explanation. That makes sense to me.

> Raft gRPC client should check proto size before sending a message
> -
>
> Key: RATIS-133
> URL: https://issues.apache.org/jira/browse/RATIS-133
> Project: Ratis
>  Issue Type: Bug
>Reporter: Mukul Kumar Singh
>Assignee: Mukul Kumar Singh
>Priority: Critical
> Attachments: RATIS-133.001.patch
>
>
> Raft client should check the entry size before the command is send, This can 
> otherwise lead to StatusRuntimeException. Checking the size on the client 
> will help avoiding error handling on the RaftServer.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-117) Add test for situation when old leader can/cannot commit log

2017-11-09 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-117:
-

Looks like the Jenkins failure is unrelated. All the tests passed in my local 
environment. +1

> Add test for situation when old leader can/cannot commit log
> 
>
> Key: RATIS-117
> URL: https://issues.apache.org/jira/browse/RATIS-117
> Project: Ratis
>  Issue Type: Test
>Reporter: Yubo Xu
>Assignee: Yubo Xu
>Priority: Minor
> Attachments: RATIS-117.001.patch, RATIS-117.002.patch, 
> RATIS-117.003.patch, RATIS-117.004.patch
>
>
> As described in Question 7 of the quiz at 
> [https://ramcloud.stanford.edu/~ongaro/userstudy/quizzes.html], an old leader 
> can complete the commitment of an old log entry satisfying certain 
> requirements. We need tests to confirm Ratis behaves correctly under such 
> cases.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-139) RaftLogWorker should use two task queues.

2017-11-08 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-139:
-

I think there're two problems here:
# when-to-sync: this is the same problem in {{FSEditLogAsync.java}} in HDFS. 
And I think we're using very similar approach in RaftLogWorker.
# the efficiency of the queue/buffer: in HDFS we need double buffer to "allow 
edits to progress concurrently to flushes without allocating new buffers each 
time." Here we actually do not have memory allocation issue. Further 
improvement may be to use a more efficient queue-like data structure to replace 
the current ArrayBlockingQueue, but I'm not sure if we should use double buffer.

BTW, for the forcesyncnum, the current hdfs has similar number which is 
EditsDoubleBuffer#initBufferSize (or FileJournalManager#outputBufferCapacity). 
Maybe we can also use the number of data size instead of log entry number as 
the indicator of forcesync#.

> RaftLogWorker should use two task queues. 
> --
>
> Key: RATIS-139
> URL: https://issues.apache.org/jira/browse/RATIS-139
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Jitendra Nath Pandey
>Assignee: Jing Zhao
>
> {code}
>   private boolean shouldFlush() {
> return pendingFlushNum >= forceSyncNum ||
> (pendingFlushNum > 0 && queue.isEmpty());
>   }
> {code}
> It is possible that queue is often non-empty, but not full to trigger force 
> sync. We could use two queues similar to two buffer approach in HDFS edit 
> logs.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-139) RaftLogWorker should use two task queues.

2017-11-08 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-139:
-

[~jnp], have you seen a performance penalty with the current implementation? 
Here the scenario is different from editlog in HDFS, since the queue is a 
pending task queue instead of a pure data buffer. If you look at the types of 
the task, all the other tasks other than writelog may trigger a sync 
automatically. And if the queue contains more writelog task, then we can wait 
till the pendingflushnum hit forcesyncnum, or till the queue is empty, or we 
hit a finalizeSegment/truncate taks. Thoughts?

> RaftLogWorker should use two task queues. 
> --
>
> Key: RATIS-139
> URL: https://issues.apache.org/jira/browse/RATIS-139
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Jitendra Nath Pandey
>
> {code}
>   private boolean shouldFlush() {
> return pendingFlushNum >= forceSyncNum ||
> (pendingFlushNum > 0 && queue.isEmpty());
>   }
> {code}
> It is possible that queue is often non-empty, but not full to trigger force 
> sync. We could use two queues similar to two buffer approach in HDFS edit 
> logs.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Assigned] (RATIS-139) RaftLogWorker should use two task queues.

2017-11-08 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-139?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao reassigned RATIS-139:
---

Assignee: Jing Zhao

> RaftLogWorker should use two task queues. 
> --
>
> Key: RATIS-139
> URL: https://issues.apache.org/jira/browse/RATIS-139
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Jitendra Nath Pandey
>Assignee: Jing Zhao
>
> {code}
>   private boolean shouldFlush() {
> return pendingFlushNum >= forceSyncNum ||
> (pendingFlushNum > 0 && queue.isEmpty());
>   }
> {code}
> It is possible that queue is often non-empty, but not full to trigger force 
> sync. We could use two queues similar to two buffer approach in HDFS edit 
> logs.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-77) "RaftServerProtocolService" and " GRpc.proto" inconsistent

2017-11-08 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-77:


It's because if the incoming request is a stream, we expect response for each 
chunk so that we can detect failure earlier.

> "RaftServerProtocolService" and " GRpc.proto" inconsistent
> --
>
> Key: RATIS-77
> URL: https://issues.apache.org/jira/browse/RATIS-77
> Project: Ratis
>  Issue Type: Bug
>Reporter: kaiyangzhang
>Assignee: garvit
>
> *Maybe a bug, look at the comments in the code and GRpc.proto*
> {code:title=GRpc.proto|borderStyle=solid}
>..
>..
>//Executes a client-streaming call , return only one response.
>rpc installSnapshot(stream ratis.common.InstallSnapshotRequestProto)
>   returns(ratis.common.InstallSnapshotReplyProto) {}
>...
>...
> {code}
> {code:title=RaftServerProtocolService.java |borderStyle=solid}
> 
> 
> @Override
>   public StreamObserver installSnapshot(
>   StreamObserver responseObserver) {
> return new StreamObserver() {
>   @Override
>   public void onNext(InstallSnapshotRequestProto request) {
> try {
>   //receive a client-streaming call, return a response stream
>   //code and GRpc.proto inconsistent
>   final InstallSnapshotReplyProto reply = 
> server.installSnapshot(request);
>   responseObserver.onNext(reply);
> } catch (Throwable e) {
>   LOG.info("{} got exception when handling installSnapshot {}: {}",
>   id, request.getServerRequest(), e);
>   responseObserver.onError(RaftGrpcUtil.wrapException(e));
> }
>   }
>..
>..
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-117) Add test for situation when old leader can/cannot commit log

2017-11-08 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-117:
-

The latest patch looks pretty good to me. Just some minors:
# Let's use "Assert.assertTrue" instead of Preconditions.assertTrue
{code}
158 Preconditions.assertTrue(logEntriesContains(followerLog, messages));
{code}
# Remove the following commented code.
{code}
174 //final RaftPeerId newLeaderId = waitForLeader(cluster).getId();
175 //// confirm the server with log is elected as new leader.
176 //
Preconditions.assertTrue(newLeaderId.equals(followerToSendLog.getId()));
177 //Thread.sleep(cluster.getMaxTimeout() + 100);
{code}
# The method {{assertLogEntriesNotMatch}}'s signature is not easy to follow. If 
you want to check whether "any log entry containing the given simple msg should 
follow certain rule", you can define a method like {{void 
checkLogEntries(RaftLog log, List expectedMessages, 
Predicate predicate)}}.
{code}
167   static void assertLogEntriesNotMatch(RaftLog log, TermIndex[] entries,
168   long startIndex, long expectedTerm, SimpleMessage... 
expectedMessages) {
{code}

> Add test for situation when old leader can/cannot commit log
> 
>
> Key: RATIS-117
> URL: https://issues.apache.org/jira/browse/RATIS-117
> Project: Ratis
>  Issue Type: Test
>Reporter: Yubo Xu
>Assignee: Yubo Xu
>Priority: Minor
> Attachments: RATIS-117.001.patch, RATIS-117.002.patch, 
> RATIS-117.003.patch
>
>
> As described in Question 7 of the quiz at 
> [https://ramcloud.stanford.edu/~ongaro/userstudy/quizzes.html], an old leader 
> can complete the commitment of an old log entry satisfying certain 
> requirements. We need tests to confirm Ratis behaves correctly under such 
> cases.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-77) "RaftServerProtocolService" and " GRpc.proto" inconsistent

2017-11-07 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-77:


I think the original comment is correct, and we should change the protobuf 
definition so that it returns a stream.

> "RaftServerProtocolService" and " GRpc.proto" inconsistent
> --
>
> Key: RATIS-77
> URL: https://issues.apache.org/jira/browse/RATIS-77
> Project: Ratis
>  Issue Type: Bug
>Reporter: kaiyangzhang
>Assignee: Jing Zhao
>
> *Maybe a bug, look at the comments in the code and GRpc.proto*
> {code:title=GRpc.proto|borderStyle=solid}
>..
>..
>//Executes a client-streaming call , return only one response.
>rpc installSnapshot(stream ratis.common.InstallSnapshotRequestProto)
>   returns(ratis.common.InstallSnapshotReplyProto) {}
>...
>...
> {code}
> {code:title=RaftServerProtocolService.java |borderStyle=solid}
> 
> 
> @Override
>   public StreamObserver installSnapshot(
>   StreamObserver responseObserver) {
> return new StreamObserver() {
>   @Override
>   public void onNext(InstallSnapshotRequestProto request) {
> try {
>   //receive a client-streaming call, return a response stream
>   //code and GRpc.proto inconsistent
>   final InstallSnapshotReplyProto reply = 
> server.installSnapshot(request);
>   responseObserver.onNext(reply);
> } catch (Throwable e) {
>   LOG.info("{} got exception when handling installSnapshot {}: {}",
>   id, request.getServerRequest(), e);
>   responseObserver.onError(RaftGrpcUtil.wrapException(e));
> }
>   }
>..
>..
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Assigned] (RATIS-77) "RaftServerProtocolService" and " GRpc.proto" inconsistent

2017-11-07 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-77?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao reassigned RATIS-77:
--

Assignee: (was: Jing Zhao)

> "RaftServerProtocolService" and " GRpc.proto" inconsistent
> --
>
> Key: RATIS-77
> URL: https://issues.apache.org/jira/browse/RATIS-77
> Project: Ratis
>  Issue Type: Bug
>Reporter: kaiyangzhang
>
> *Maybe a bug, look at the comments in the code and GRpc.proto*
> {code:title=GRpc.proto|borderStyle=solid}
>..
>..
>//Executes a client-streaming call , return only one response.
>rpc installSnapshot(stream ratis.common.InstallSnapshotRequestProto)
>   returns(ratis.common.InstallSnapshotReplyProto) {}
>...
>...
> {code}
> {code:title=RaftServerProtocolService.java |borderStyle=solid}
> 
> 
> @Override
>   public StreamObserver installSnapshot(
>   StreamObserver responseObserver) {
> return new StreamObserver() {
>   @Override
>   public void onNext(InstallSnapshotRequestProto request) {
> try {
>   //receive a client-streaming call, return a response stream
>   //code and GRpc.proto inconsistent
>   final InstallSnapshotReplyProto reply = 
> server.installSnapshot(request);
>   responseObserver.onNext(reply);
> } catch (Throwable e) {
>   LOG.info("{} got exception when handling installSnapshot {}: {}",
>   id, request.getServerRequest(), e);
>   responseObserver.onError(RaftGrpcUtil.wrapException(e));
> }
>   }
>..
>..
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-129) Compile protobuf and shade if the shaded source directory is missing

2017-11-03 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-129:
-

+1

> Compile protobuf and shade if the shaded source directory is missing
> 
>
> Key: RATIS-129
> URL: https://issues.apache.org/jira/browse/RATIS-129
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: r129_20171103.patch
>
>
> Currently, we uses skipShade to activate protobuf compilation and shading.  
> It is better to check if the corresponding shaded source directory is missing.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-117) Add test for situation when old leader can/cannot commit log

2017-11-03 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-117:
-

Looks like TestRaftWithSimulatedRpc fails consistently. Will do some debugging.

> Add test for situation when old leader can/cannot commit log
> 
>
> Key: RATIS-117
> URL: https://issues.apache.org/jira/browse/RATIS-117
> Project: Ratis
>  Issue Type: Test
>Reporter: Yubo Xu
>Assignee: Yubo Xu
>Priority: Minor
> Attachments: RATIS-117.001.patch, RATIS-117.002.patch
>
>
> As described in Question 7 of the quiz at 
> [https://ramcloud.stanford.edu/~ongaro/userstudy/quizzes.html], an old leader 
> can complete the commitment of an old log entry satisfying certain 
> requirements. We need tests to confirm Ratis behaves correctly under such 
> cases.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-102) Clean generated sources as part of the default clean lifecycle

2017-11-03 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-102:
-

Looks like with ratis-102, if I run "mvn package -DskipTests" and then "mvn 
test" on a clean repo, I will get a lot of compilation errors. And if I revert 
ratis-102, everything works fine. Could you please take a look, [~szetszwo]? 

> Clean generated sources as part of the default clean lifecycle
> --
>
> Key: RATIS-102
> URL: https://issues.apache.org/jira/browse/RATIS-102
> Project: Ratis
>  Issue Type: Bug
>Reporter: Elek, Marton
>Assignee: Elek, Marton
>Priority: Major
>  Labels: build
> Fix For: 0.2.0-alpha
>
> Attachments: RATIS-102.000.patch, RATIS-102.001.patch, 
> RATIS-102.002.patch, RATIS-102.003.patch
>
>
> RATIS-49 introduced new profiles to cleanup the generated sources/proto files 
> in the shaded artifacts.
> I suggest to make it more easier by binding the additional {clean:clean} 
> plugin calls to the clean phase of the default clean lifecycle instead of 
> trigger them from a separated profile.  
> In RATIS-4 I experimenting  with build scripts and yetus test-patch script. 
> As the simple {{mvn clean}} command is more common, it would be easier to 
> switch to the simple clean without the profile.
> The cleanup could be done with triggering additional clean plugin execution.
> To test:
> {code}
> git checkout 52c4b64
> mvn clean package -DskipTests
> git checkout master
> mvn clean package -DskipTests
> {code}
> Without the patch the second only works with -Pclean-shade, with the proposed 
> patch it works without activating any additional profile



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Resolved] (RATIS-128) Missing maven plugin version for protobuf and shade plugins

2017-11-03 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-128?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-128.
-
   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

I've committed the patch. Thanks for the fix, Nicholas!

> Missing maven plugin version for protobuf and shade plugins
> ---
>
> Key: RATIS-128
> URL: https://issues.apache.org/jira/browse/RATIS-128
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Fix For: 0.2.0-alpha
>
> Attachments: r128_20171102.patch
>
>
> {code}
> [WARNING] 'build.plugins.plugin.version' for 
> org.apache.maven.plugins:maven-shade-plugin is missing. @ 
> org.apache.ratis:ratis-hadoop-shaded:[unknown-version], 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-hadoop-shaded/pom.xml, line 184, 
> column 19
> [WARNING] 
> [WARNING] Some problems were encountered while building the effective model 
> for org.apache.ratis:ratis-proto-shaded:jar:0.1.1-alpha-SNAPSHOT
> [WARNING] 'build.plugins.plugin.version' for 
> org.xolstice.maven.plugins:protobuf-maven-plugin is missing. @ 
> org.apache.ratis:ratis-proto-shaded:[unknown-version], 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-proto-shaded/pom.xml, line 491, 
> column 19
> [WARNING] 'build.plugins.plugin.version' for 
> org.apache.maven.plugins:maven-shade-plugin is missing. @ 
> org.apache.ratis:ratis-proto-shaded:[unknown-version], 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-proto-shaded/pom.xml, line 524, 
> column 19
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-128) Missing maven plugin version for protobuf and shade plugins

2017-11-03 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-128:
-

+1. I will commit the patch shortly.

> Missing maven plugin version for protobuf and shade plugins
> ---
>
> Key: RATIS-128
> URL: https://issues.apache.org/jira/browse/RATIS-128
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: r128_20171102.patch
>
>
> {code}
> [WARNING] 'build.plugins.plugin.version' for 
> org.apache.maven.plugins:maven-shade-plugin is missing. @ 
> org.apache.ratis:ratis-hadoop-shaded:[unknown-version], 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-hadoop-shaded/pom.xml, line 184, 
> column 19
> [WARNING] 
> [WARNING] Some problems were encountered while building the effective model 
> for org.apache.ratis:ratis-proto-shaded:jar:0.1.1-alpha-SNAPSHOT
> [WARNING] 'build.plugins.plugin.version' for 
> org.xolstice.maven.plugins:protobuf-maven-plugin is missing. @ 
> org.apache.ratis:ratis-proto-shaded:[unknown-version], 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-proto-shaded/pom.xml, line 491, 
> column 19
> [WARNING] 'build.plugins.plugin.version' for 
> org.apache.maven.plugins:maven-shade-plugin is missing. @ 
> org.apache.ratis:ratis-proto-shaded:[unknown-version], 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-proto-shaded/pom.xml, line 524, 
> column 19
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-121) In RaftServer.Builder, allow serverId and group to be initialized automatically

2017-10-25 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-121:
-

bq. This patch just sets the default to host:port

hmm, I guess I missed that part of code... Then everything makes sense to me. +1

> In RaftServer.Builder, allow serverId and group to be initialized 
> automatically
> ---
>
> Key: RATIS-121
> URL: https://issues.apache.org/jira/browse/RATIS-121
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r121_20171013.patch
>
>
> When the serverId is missing, it can be initialized automatically using the 
> given properties such as host and port.
> Also, the group should be automatically initialized to an empty group since 
> group may not be known when starting a server.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-121) In RaftServer.Builder, allow serverId and group to be initialized automatically

2017-10-25 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-121:
-

bq. I found that we don't really need unique raft server IDs since we already 
have address (i.e. host:post)

Yes, host:port can be used to identify a peer in most of the cases. However, I 
think it's better to have a unique peer id in the protocol which is not coupled 
with any lower level implementations. I.e., we allow upper layer application to 
generate peer IDs based on host:port, or to use other ways, but in Ratis we 
only need to accept the passed-in IDs.

> In RaftServer.Builder, allow serverId and group to be initialized 
> automatically
> ---
>
> Key: RATIS-121
> URL: https://issues.apache.org/jira/browse/RATIS-121
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r121_20171013.patch
>
>
> When the serverId is missing, it can be initialized automatically using the 
> given properties such as host and port.
> Also, the group should be automatically initialized to an empty group since 
> group may not be known when starting a server.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-119) RaftServerImpl.registerMBean may throw MalformedObjectNameException

2017-10-24 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-119:
-

+1

> RaftServerImpl.registerMBean may throw MalformedObjectNameException
> ---
>
> Key: RATIS-119
> URL: https://issues.apache.org/jira/browse/RATIS-119
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Minor
> Attachments: r119_20171018.patch, r119_20171019.patch, 
> r119_20171024.patch
>
>
> [~linyiqun] has reported that RaftServerImpl.registerMBean may throw 
> MalformedObjectNameException in HDFS-12593.
> {code}
> 2017-10-10 14:50:01,163 [Datanode State Machine Thread - 0] ERROR 
> impl.RaftServerImpl (RaftServerImpl.java:registerMBean(182)) - RaftServer JMX 
> bean can't be registered
> javax.management.MalformedObjectNameException: Invalid character ':' in value 
> part of property
>   at javax.management.ObjectName.construct(ObjectName.java:618)
>   at javax.management.ObjectName.(ObjectName.java:1382)
>   at 
> org.apache.ratis.server.impl.RaftServerImpl.registerMBean(RaftServerImpl.java:179)
>   ...
>   at 
> org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.start(DatanodeStateMachine.java:126)
>   at 
> org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.lambda$0(DatanodeStateMachine.java:280)
>   at java.lang.Thread.run(Thread.java:745)
> {code}
> This is probably due to HDFS using host:port as raft server id.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-121) In RaftServer.Builder, allow serverId and group to be initialized automatically

2017-10-24 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-121:
-

If the serverId is not specified directly, I think the application (that uses 
Ratis library) should generate the server ID based on some rules, such as using 
host+port. Therefore I'm not very sure whether we need to change all the server 
id into an id supplier. Considering id is necessary for the whole protocol, we 
should enforce the upper layer application to provide server id in the very 
beginning. Thoughts?

> In RaftServer.Builder, allow serverId and group to be initialized 
> automatically
> ---
>
> Key: RATIS-121
> URL: https://issues.apache.org/jira/browse/RATIS-121
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r121_20171013.patch
>
>
> When the serverId is missing, it can be initialized automatically using the 
> given properties such as host and port.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-120) In RaftClient.Builder, allow clientRpc to be initialized automatically

2017-10-24 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-120:
-

+1

> In RaftClient.Builder, allow clientRpc to be initialized automatically 
> ---
>
> Key: RATIS-120
> URL: https://issues.apache.org/jira/browse/RATIS-120
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r120_20171013.patch
>
>
> Current RaftClient.Builder requires user to initial clientRpc explicitly.  It 
> is cleaner if clientRpc can be initialized automatically.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-119) RaftServerImpl.registerMBean may throw MalformedObjectNameException

2017-10-24 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-119:
-

The patch looks great to me. Some minors:
# # For {{public synchronized boolean register(Object mBean, 
Supplier... names) }}, we can change Supplier... into 
Collection to fix the "possible heap pollution" warning.
# Can we combine the LifeCycle with RaftServerImpl#RaftServerJmxAdapter, 
instead of directly inside of JmxRegister? Feels like LifeCycle is more tightly 
coupled with RaftServerImpl.


> RaftServerImpl.registerMBean may throw MalformedObjectNameException
> ---
>
> Key: RATIS-119
> URL: https://issues.apache.org/jira/browse/RATIS-119
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Minor
> Attachments: r119_20171018.patch, r119_20171019.patch
>
>
> [~linyiqun] has reported that RaftServerImpl.registerMBean may throw 
> MalformedObjectNameException in HDFS-12593.
> {code}
> 2017-10-10 14:50:01,163 [Datanode State Machine Thread - 0] ERROR 
> impl.RaftServerImpl (RaftServerImpl.java:registerMBean(182)) - RaftServer JMX 
> bean can't be registered
> javax.management.MalformedObjectNameException: Invalid character ':' in value 
> part of property
>   at javax.management.ObjectName.construct(ObjectName.java:618)
>   at javax.management.ObjectName.(ObjectName.java:1382)
>   at 
> org.apache.ratis.server.impl.RaftServerImpl.registerMBean(RaftServerImpl.java:179)
>   ...
>   at 
> org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.start(DatanodeStateMachine.java:126)
>   at 
> org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.lambda$0(DatanodeStateMachine.java:280)
>   at java.lang.Thread.run(Thread.java:745)
> {code}
> This is probably due to HDFS using host:port as raft server id.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-117) Add test for situation when old leader can/cannot commit log

2017-10-03 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-117:
-

Thanks for adding the new tests, [~yubox]! The patch looks good to me after 
fixing the following:
# The following code will block the test: 3 followers may have been killed 
already, and the {{client.send}} call will be blocked while waiting for 
committing the msg. So we need to run the following code in a new thread.
{code}
139 SimpleMessage[] messages = SimpleMessage.create(1);
140 try(final RaftClient client = cluster.createClient(leaderId)) {
141   for (SimpleMessage message: messages) {
142 client.send(message);
143   }
144 }
{code}
# "log.getEntries(1, Long.MAX_VALUE)" should be "log.getEntries(1, 2)": besides 
the client msg, the log entries should also include a placeholder entry for the 
new leader.
{code}
163 .forEach(log -> RaftTestUtil.assertLogEntries(log,
164 log.getEntries(1, Long.MAX_VALUE), 1, term, 
messages));
{code}

> Add test for situation when old leader can/cannot commit log
> 
>
> Key: RATIS-117
> URL: https://issues.apache.org/jira/browse/RATIS-117
> Project: Ratis
>  Issue Type: Test
>Reporter: Yubo Xu
>Assignee: Yubo Xu
>Priority: Minor
> Attachments: RATIS-117.001.patch
>
>
> As described in Question 7 of the quiz at 
> [https://ramcloud.stanford.edu/~ongaro/userstudy/quizzes.html], an old leader 
> can complete the commitment of an old log entry satisfying certain 
> requirements. We need tests to confirm Ratis behaves correctly under such 
> cases.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-116) In PendingRequests, the requests are never removed from the map

2017-09-28 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-116:
-

+1

> In PendingRequests, the requests are never removed from the map
> ---
>
> Key: RATIS-116
> URL: https://issues.apache.org/jira/browse/RATIS-116
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r116_20170922.patch
>
>
> xmtsui has reported that there is a memory leak problem in 
> PendingRequests.java
> The field pendingRequests, can only be added, but no remove logic.
> See https://github.com/hortonworks/ratis/issues/7



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-118) Add missing license headers for the ServerInformation feature

2017-09-28 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-118:
-

+1

> Add missing license headers for the ServerInformation feature
> -
>
> Key: RATIS-118
> URL: https://issues.apache.org/jira/browse/RATIS-118
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r118_20170928.patch
>
>
> - A few ServerInformation related files does not have a license header.
> - The shaded sources in ratis-hadoop-shaded should be excluded from rat, i.e. 
> license header check.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-107) Restrict reinitialize to be from/to an empty group

2017-09-14 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-107:
-

I think one state machine managing all the storage containers can be the case 
for ozone, but not a general case. For example, if we want to use multiple raft 
groups to replicate different shards for a database, we can have multiple state 
machines there. In that case, {{reinitialize}} as a RPC cannot access/include 
all the information we want, and we should not trigger the new raft group 
creation from the raft protocol level.

I will finish a partial patch these days which converts the reinitialize to a 
local API. 

> Restrict reinitialize to be from/to an empty group
> --
>
> Key: RATIS-107
> URL: https://issues.apache.org/jira/browse/RATIS-107
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r107_20170816.patch, r107_20170822.patch, 
> r107_20170909.patch
>
>
> In order to avoid servers being accidentally reinitialized incorrectly, we 
> propose that reinitialize must be applied from an empty group to a non-empty 
> group, or from a non-empty group to an empty group.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-114) TestRaftWithHadoopRpc.testWithLoad may timeout

2017-09-06 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-114:
-

Is this because the retry policy of the hadoop rpc client?

> TestRaftWithHadoopRpc.testWithLoad may timeout
> --
>
> Key: RATIS-114
> URL: https://issues.apache.org/jira/browse/RATIS-114
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>
> It may stop having any progress at some index until timeout.  For example, 
> see [this 
> log|https://issues.apache.org/jira/secure/attachment/12885286/org.apache.ratis.hadooprpc.TestRaftWithHadoopRpc-output.txt].



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-111) RaftLogWorker may throw IllegalStateException

2017-09-06 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-111:
-

Yes the change of the patch is good. We can commit it first and fix the timeout 
in separate jira. +1

> RaftLogWorker may throw IllegalStateException
> -
>
> Key: RATIS-111
> URL: https://issues.apache.org/jira/browse/RATIS-111
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: 
> org.apache.ratis.hadooprpc.TestRaftWithHadoopRpc-output.txt, 
> org.apache.ratis.server.simulation.TestNotLeaderExceptionWithSimulation-output.txt,
>  r111_20170823.patch, r111_20170824b.patch, r111_20170824c.patch, 
> r111_20170824.patch, r111_20170828.patch, r111_20170829.patch
>
>
> {code}
> Exception in thread "RaftLogWorker for Storage Directory 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B"
>  2017-08-22 15:52:47,983 INFO  impl.RaftServerImpl 
> (RaftLogWorker.java:execute(278)) - RaftLogWorker-s4 finalizing log segment 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s4/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B/current/log_inprogress_0
> org.apache.ratis.util.ExitUtils$ExitException: RaftLogWorker for Storage 
> Directory 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B
>  failed.
>   at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:88)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:185)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: File 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B/current/log_inprogress_0
>  does not exist.
>   at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker$FinalizeLogSegment.execute(RaftLogWorker.java:280)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:155)
>   ... 1 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-107) Restrict reinitialize to be from/to an empty group

2017-09-04 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-107:
-

Thanks for working on this, Nicholas. Here I think we may want to change the 
{{reinitialize}} RPC into a local API which is triggered by the local 
upper-level application code. The raft groups are usually determined by the 
application (e.g., ozone) based on their requirement, and each decision should 
indicates: 1) the state machine for replication (e.g., which storage 
containers), and 2) the nodes to do the replication. A RPC call in the raft 
layer, like the current {{reinitialize}} can only tell the information about 
#2. We thus actually need a RPC call in the application layer, whose server 
side implementation calls the local RaftServerProxy's API to add the local node 
to a new/existing raft group.

What do you think?

> Restrict reinitialize to be from/to an empty group
> --
>
> Key: RATIS-107
> URL: https://issues.apache.org/jira/browse/RATIS-107
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r107_20170816.patch, r107_20170822.patch
>
>
> In order to avoid servers being accidentally reinitialized incorrectly, we 
> propose that reinitialize must be applied from an empty group to a non-empty 
> group, or from a non-empty group to an empty group.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-102) Clean generated sources as part of the default clean lifecycle

2017-09-04 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-102:
-

The patch looks good to me. +1

Do you also want to take a look at the patch, [~szetszwo]?

> Clean generated sources as part of the default clean lifecycle
> --
>
> Key: RATIS-102
> URL: https://issues.apache.org/jira/browse/RATIS-102
> Project: Ratis
>  Issue Type: Bug
>Reporter: Elek, Marton
>Assignee: Elek, Marton
>  Labels: build
> Attachments: RATIS-102.000.patch
>
>
> RATIS-49 introduced new profiles to cleanup the generated sources/proto files 
> in the shaded artifacts.
> I suggest to make it more easier by binding the additional {clean:clean} 
> plugin calls to the clean phase of the default clean lifecycle instead of 
> trigger them from a separated profile.  
> In RATIS-4 I experimenting  with build scripts and yetus test-patch script. 
> As the simple {{mvn clean}} command is more common, it would be easier to 
> switch to the simple clean without the profile.
> The cleanup could be done with triggering additional clean plugin execution.
> To test:
> {code}
> git checkout 52c4b64
> mvn clean package -DskipTests
> git checkout master
> mvn clean package -DskipTests
> {code}
> Without the patch the second only works with -Pclean-shade, with the proposed 
> patch it works without activating any additional profile



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (RATIS-111) RaftLogWorker may throw IllegalStateException

2017-09-04 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-111?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-111:

Attachment: org.apache.ratis.hadooprpc.TestRaftWithHadoopRpc-output.txt

Upload the log file. Looks like in {{testWithLoad}} the log index kept 
increasing until 4193, thus the failure might be simply because the timeout 
value is too small.

> RaftLogWorker may throw IllegalStateException
> -
>
> Key: RATIS-111
> URL: https://issues.apache.org/jira/browse/RATIS-111
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: 
> org.apache.ratis.hadooprpc.TestRaftWithHadoopRpc-output.txt, 
> org.apache.ratis.server.simulation.TestNotLeaderExceptionWithSimulation-output.txt,
>  r111_20170823.patch, r111_20170824b.patch, r111_20170824c.patch, 
> r111_20170824.patch, r111_20170828.patch, r111_20170829.patch
>
>
> {code}
> Exception in thread "RaftLogWorker for Storage Directory 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B"
>  2017-08-22 15:52:47,983 INFO  impl.RaftServerImpl 
> (RaftLogWorker.java:execute(278)) - RaftLogWorker-s4 finalizing log segment 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s4/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B/current/log_inprogress_0
> org.apache.ratis.util.ExitUtils$ExitException: RaftLogWorker for Storage 
> Directory 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B
>  failed.
>   at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:88)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:185)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: File 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B/current/log_inprogress_0
>  does not exist.
>   at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker$FinalizeLogSegment.execute(RaftLogWorker.java:280)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:155)
>   ... 1 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (RATIS-111) RaftLogWorker may throw IllegalStateException

2017-09-04 Thread Jing Zhao (JIRA)

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

Jing Zhao edited comment on RATIS-111 at 9/4/17 10:34 PM:
--

Thanks for the fix, Nicholas! The patch looks good to me. But 
{{TestRaftWithHadoopRpc#testWithLoad}} failed with timeout in my local 
environment. Could you please take another look?


was (Author: jingzhao):
Thanks for the fix, Nicholas! The patch looks good to me. But 
{{TestRaftWithHadoopRpc#testWithLoad}} failed with timeout while running tests 
in my local environment. Could you please take another look?

> RaftLogWorker may throw IllegalStateException
> -
>
> Key: RATIS-111
> URL: https://issues.apache.org/jira/browse/RATIS-111
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: 
> org.apache.ratis.server.simulation.TestNotLeaderExceptionWithSimulation-output.txt,
>  r111_20170823.patch, r111_20170824b.patch, r111_20170824c.patch, 
> r111_20170824.patch, r111_20170828.patch, r111_20170829.patch
>
>
> {code}
> Exception in thread "RaftLogWorker for Storage Directory 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B"
>  2017-08-22 15:52:47,983 INFO  impl.RaftServerImpl 
> (RaftLogWorker.java:execute(278)) - RaftLogWorker-s4 finalizing log segment 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s4/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B/current/log_inprogress_0
> org.apache.ratis.util.ExitUtils$ExitException: RaftLogWorker for Storage 
> Directory 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B
>  failed.
>   at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:88)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:185)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: File 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B/current/log_inprogress_0
>  does not exist.
>   at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker$FinalizeLogSegment.execute(RaftLogWorker.java:280)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:155)
>   ... 1 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-111) RaftLogWorker may throw IllegalStateException

2017-09-04 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-111:
-

Thanks for the fix, Nicholas! The patch looks good to me. But 
{{TestRaftWithHadoopRpc#testWithLoad}} failed with timeout while running tests 
in my local environment. Could you please take another look?

> RaftLogWorker may throw IllegalStateException
> -
>
> Key: RATIS-111
> URL: https://issues.apache.org/jira/browse/RATIS-111
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: 
> org.apache.ratis.server.simulation.TestNotLeaderExceptionWithSimulation-output.txt,
>  r111_20170823.patch, r111_20170824b.patch, r111_20170824c.patch, 
> r111_20170824.patch, r111_20170828.patch, r111_20170829.patch
>
>
> {code}
> Exception in thread "RaftLogWorker for Storage Directory 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B"
>  2017-08-22 15:52:47,983 INFO  impl.RaftServerImpl 
> (RaftLogWorker.java:execute(278)) - RaftLogWorker-s4 finalizing log segment 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s4/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B/current/log_inprogress_0
> org.apache.ratis.util.ExitUtils$ExitException: RaftLogWorker for Storage 
> Directory 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B
>  failed.
>   at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:88)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:185)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: File 
> /Users/szetszwo/hadoop/incubator-ratis/ratis-server/target/test/data/e19600c7a0228b58/MiniRaftClusterWithSimulatedRpc/s3/group-E1192218-3981-4FC5-90BF-4CFB0D270F6B/current/log_inprogress_0
>  does not exist.
>   at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker$FinalizeLogSegment.execute(RaftLogWorker.java:280)
>   at 
> org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:155)
>   ... 1 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-105) Server should check group id for client requests

2017-09-04 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-105:
-

The patch looks good to me, except we need to rename the java files of 
RaftNotLeaderExceptionBaseTest and its subclasses. Since the change is trivial, 
I will do it while committing the patch.

> Server should check group id for client requests 
> -
>
> Key: RATIS-105
> URL: https://issues.apache.org/jira/browse/RATIS-105
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 0.2.0-alpha
>
> Attachments: r105_20170814.patch, r105_20170815.patch, 
> r105_20170821.patch
>
>
> In RATIS-100, we found a bug that a server may response to another server 
> with different group so that a cluster with multiple groups may not work 
> correctly.  The solution is to check the group id for each server request 
> before responding to it.
> In this JIRA, we add a similar group id check for the client requests.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-100) Fix bugs for running multiple raft groups with a state machine

2017-09-04 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-100:
-

The patch needs a minor fix to compile:
{code}
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java 
b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
index b227e47..7e08809 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
@@ -80,7 +80,7 @@ public abstract class RaftBasicTests extends BaseTest {
 LOG.info(cluster.printServers());
 
 final SimpleMessage[] messages = SimpleMessage.create(10);
-try(final RaftClient client = cluster.createClient(null)) {
+try(final RaftClient client = cluster.createClient()) {
   for (SimpleMessage message : messages) {
 client.send(message);
   }
@@ -149,7 +149,7 @@ public abstract class RaftBasicTests extends BaseTest {
 
 final List clients
 = Stream.iterate(0, i -> i+1).limit(numClients)
-.map(i -> cluster.createClient(null))
+.map(i -> cluster.createClient())
 .map(c -> new Client4TestWithLoad(c, numMessages))
 .collect(Collectors.toList());
 clients.forEach(Thread::start);
diff --git 
a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
 
b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
index f41e764..73ce69d 100644
--- 
a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
+++ 
b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
@@ -156,7 +156,7 @@ public class TestStateMachine extends BaseTest {
 
 int numTrx = 100;
 final RaftTestUtil.SimpleMessage[] messages = 
RaftTestUtil.SimpleMessage.create(numTrx);
-try(final RaftClient client = cluster.createClient(null)) {
+try(final RaftClient client = cluster.createClient()) {
   for (RaftTestUtil.SimpleMessage message : messages) {
 client.send(message);
   }
{code}

I will commit the patch along with the fix.

> Fix bugs for running multiple raft groups with a state machine
> --
>
> Key: RATIS-100
> URL: https://issues.apache.org/jira/browse/RATIS-100
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r100_20170804.patch, r100_20170809b.patch, 
> r100_20170809c.patch, r100_20170809.patch, r100_20170810.patch, 
> r100_20170811.patch, r100_20170821b.patch, r100_20170821.patch, 
> r100_no_leader_case.log
>
>
> We found the following bugs when trying to add a test similar to 
> ReinitializationBaseTest.runTestReinitializeMultiGroups(..) with a state 
> machine.
> - In PendingRequests, the {{last}} PendingRequest is not updated in 
> addConfRequest(..).
> - In RaftServerImpl, it should check if the group in the request is the same 
> as the group in the server.
> - In StateMachineUpdater, it should join the updater thread in stop().



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Issue Comment Deleted] (RATIS-105) Server should check group id for client requests

2017-09-04 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-105:

Comment: was deleted

(was: I've committed the patch. Thanks Nicholas for the contribution!)

> Server should check group id for client requests 
> -
>
> Key: RATIS-105
> URL: https://issues.apache.org/jira/browse/RATIS-105
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 0.2.0-alpha
>
> Attachments: r105_20170814.patch, r105_20170815.patch, 
> r105_20170821.patch
>
>
> In RATIS-100, we found a bug that a server may response to another server 
> with different group so that a cluster with multiple groups may not work 
> correctly.  The solution is to check the group id for each server request 
> before responding to it.
> In this JIRA, we add a similar group id check for the client requests.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Issue Comment Deleted] (RATIS-105) Server should check group id for client requests

2017-09-04 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-105:

Comment: was deleted

(was: The patch needs a minor fix to compile:
{code}
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java 
b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
index b227e47..7e08809 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
@@ -80,7 +80,7 @@ public abstract class RaftBasicTests extends BaseTest {
 LOG.info(cluster.printServers());
 
 final SimpleMessage[] messages = SimpleMessage.create(10);
-try(final RaftClient client = cluster.createClient(null)) {
+try(final RaftClient client = cluster.createClient()) {
   for (SimpleMessage message : messages) {
 client.send(message);
   }
@@ -149,7 +149,7 @@ public abstract class RaftBasicTests extends BaseTest {
 
 final List clients
 = Stream.iterate(0, i -> i+1).limit(numClients)
-.map(i -> cluster.createClient(null))
+.map(i -> cluster.createClient())
 .map(c -> new Client4TestWithLoad(c, numMessages))
 .collect(Collectors.toList());
 clients.forEach(Thread::start);
diff --git 
a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
 
b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
index f41e764..73ce69d 100644
--- 
a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
+++ 
b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
@@ -156,7 +156,7 @@ public class TestStateMachine extends BaseTest {
 
 int numTrx = 100;
 final RaftTestUtil.SimpleMessage[] messages = 
RaftTestUtil.SimpleMessage.create(numTrx);
-try(final RaftClient client = cluster.createClient(null)) {
+try(final RaftClient client = cluster.createClient()) {
   for (RaftTestUtil.SimpleMessage message : messages) {
 client.send(message);
   }
{code}

I will commit the patch along with the fix.)

> Server should check group id for client requests 
> -
>
> Key: RATIS-105
> URL: https://issues.apache.org/jira/browse/RATIS-105
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 0.2.0-alpha
>
> Attachments: r105_20170814.patch, r105_20170815.patch, 
> r105_20170821.patch
>
>
> In RATIS-100, we found a bug that a server may response to another server 
> with different group so that a cluster with multiple groups may not work 
> correctly.  The solution is to check the group id for each server request 
> before responding to it.
> In this JIRA, we add a similar group id check for the client requests.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Resolved] (RATIS-105) Server should check group id for client requests

2017-09-04 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-105.
-
   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

I've committed the patch. Thanks Nicholas for the contribution!

> Server should check group id for client requests 
> -
>
> Key: RATIS-105
> URL: https://issues.apache.org/jira/browse/RATIS-105
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 0.2.0-alpha
>
> Attachments: r105_20170814.patch, r105_20170815.patch, 
> r105_20170821.patch
>
>
> In RATIS-100, we found a bug that a server may response to another server 
> with different group so that a cluster with multiple groups may not work 
> correctly.  The solution is to check the group id for each server request 
> before responding to it.
> In this JIRA, we add a similar group id check for the client requests.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-100) Fix bugs for running multiple raft groups with a state machine

2017-09-04 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-100:
-

+1. I will commit the patch shortly,

> Fix bugs for running multiple raft groups with a state machine
> --
>
> Key: RATIS-100
> URL: https://issues.apache.org/jira/browse/RATIS-100
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r100_20170804.patch, r100_20170809b.patch, 
> r100_20170809c.patch, r100_20170809.patch, r100_20170810.patch, 
> r100_20170811.patch, r100_20170821b.patch, r100_20170821.patch, 
> r100_no_leader_case.log
>
>
> We found the following bugs when trying to add a test similar to 
> ReinitializationBaseTest.runTestReinitializeMultiGroups(..) with a state 
> machine.
> - In PendingRequests, the {{last}} PendingRequest is not updated in 
> addConfRequest(..).
> - In RaftServerImpl, it should check if the group in the request is the same 
> as the group in the server.
> - In StateMachineUpdater, it should join the updater thread in stop().



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Resolved] (RATIS-97) Leader may not have majority after setConfiguration

2017-08-10 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-97?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-97.

   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

I've committed the patch. Thanks Nicholas!

> Leader may not have majority after setConfiguration
> ---
>
> Key: RATIS-97
> URL: https://issues.apache.org/jira/browse/RATIS-97
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 0.2.0-alpha
>
> Attachments: r97_20170728.patch, r97_20170731.patch, 
> r97_20170801b.patch, r97_20170801.patch, r97_20170807.patch
>
>
> Here is an example: Suppose a group has 3 servers and one of them is the 
> leader.  Use setConfiguration to add 6 more servers.  The leader is still 
> running as a leader although it does not has majority anymore.  The 6 new 
> servers may elect a new leader among them.  Finally, there are two leaders in 
> the group.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-97) Leader may not have majority after setConfiguration

2017-08-10 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-97:


+1. I will commit the patch shortly.

> Leader may not have majority after setConfiguration
> ---
>
> Key: RATIS-97
> URL: https://issues.apache.org/jira/browse/RATIS-97
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r97_20170728.patch, r97_20170731.patch, 
> r97_20170801b.patch, r97_20170801.patch, r97_20170807.patch
>
>
> Here is an example: Suppose a group has 3 servers and one of them is the 
> leader.  Use setConfiguration to add 6 more servers.  The leader is still 
> running as a leader although it does not has majority anymore.  The 6 new 
> servers may elect a new leader among them.  Finally, there are two leaders in 
> the group.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Resolved] (RATIS-94) Expose basic information over JMX

2017-08-10 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-94?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-94.

   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

I've committed the patch. Thanks for the contribution, [~elek]!

> Expose basic information over JMX
> -
>
> Key: RATIS-94
> URL: https://issues.apache.org/jira/browse/RATIS-94
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Elek, Marton
>Assignee: Elek, Marton
> Fix For: 0.2.0-alpha
>
> Attachments: RATIS-94-1.patch, RATIS-94.2.patch, RATIS-94.3.patch, 
> RATIS-94.4.patch
>
>
> To make it easier to debug the current state of the nodes the basic 
> RatisServer information should be exposed over the JMX interface. Such as: 
> role (leader,follower), latest term, index, follower peers (in case of LEADER)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-94) Expose basic information over JMX

2017-08-07 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-94:


Besides, you may need to rebase the patch... Thanks.

> Expose basic information over JMX
> -
>
> Key: RATIS-94
> URL: https://issues.apache.org/jira/browse/RATIS-94
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Elek, Marton
>Assignee: Elek, Marton
> Attachments: RATIS-94-1.patch, RATIS-94.2.patch, RATIS-94.3.patch
>
>
> To make it easier to debug the current state of the nodes the basic 
> RatisServer information should be exposed over the JMX interface. Such as: 
> role (leader,follower), latest term, index, follower peers (in case of LEADER)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-94) Expose basic information over JMX

2017-08-07 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-94:


Yes RaftPeer may be better. BTW,
{code}
972   .map(leaderState1 ->
973   leaderState.getFollowers().stream()
{code}
Here on line 973 leaderState should be leaderState1? I suggest to rename 
leaderState1 to leader.

Other than this the patch looks good to me. +1 after addressing the comments.

> Expose basic information over JMX
> -
>
> Key: RATIS-94
> URL: https://issues.apache.org/jira/browse/RATIS-94
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Elek, Marton
>Assignee: Elek, Marton
> Attachments: RATIS-94-1.patch, RATIS-94.2.patch, RATIS-94.3.patch
>
>
> To make it easier to debug the current state of the nodes the basic 
> RatisServer information should be exposed over the JMX interface. Such as: 
> role (leader,follower), latest term, index, follower peers (in case of LEADER)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-97) Leader may not have majority after setConfiguration

2017-08-04 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-97:


Thanks for the patch, Nicholas! Looks like we may have two scenarios here:
# There are old members already in the RaftGroup, and we're adding new members. 
For old members this whole change is actually a re-configuration call. The 
original leader will understand who are the members in the old/new 
configuration. For new members adding to the raft group, initially they only 
need to understand that they now should handle requests belonging to this new 
raft group (which means they are able to map incoming requests to the 
corresponding raft group based on the group id information). Then while the 
original leader trying to commit the conf(old, new) entry, these new members 
will have the chance to learn all the old/new members in this group. I.e., the 
knowledge about the old and new configuration should be learnt through the 
conf(old, new) log entry instead of passed from the {{reinitialize}} API.
# We're creating a brand new raft group. In this case we can pass in the 
initial raft configuration information through {{reinitialize}} (or a new 
{{attendGroup}} API in the future).

Therefore I think the {{reinitialize}} API should first take {{RaftGroupId}} as 
its input. Then for scenario #1, we do not need to pass the group member 
information in this API. For scenario #2, we pass in the initial members of the 
group. What do you think?

> Leader may not have majority after setConfiguration
> ---
>
> Key: RATIS-97
> URL: https://issues.apache.org/jira/browse/RATIS-97
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r97_20170728.patch, r97_20170731.patch, 
> r97_20170801b.patch, r97_20170801.patch
>
>
> Here is an example: Suppose a group has 3 servers and one of them is the 
> leader.  Use setConfiguration to add 6 more servers.  The leader is still 
> running as a leader although it does not has majority anymore.  The 6 new 
> servers may elect a new leader among them.  Finally, there are two leaders in 
> the group.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Resolved] (RATIS-96) LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException

2017-08-01 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-96?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-96.

   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

I've committed the patch. Thanks for the fix, Nicholas!

> LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException
> -
>
> Key: RATIS-96
> URL: https://issues.apache.org/jira/browse/RATIS-96
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 0.2.0-alpha
>
> Attachments: r96_20170720.patch, r96_20170725.patch, 
> r96_20170727.patch, r96_20170731.patch
>
>
> {code}
> java.lang.ArrayIndexOutOfBoundsException: 0
>   at 
> org.apache.ratis.server.impl.LeaderState.computeLastCommitted(LeaderState.java:490)
>   at 
> org.apache.ratis.server.impl.LeaderState.updateLastCommitted(LeaderState.java:400)
>   at 
> org.apache.ratis.server.impl.LeaderState.handleEvent(LeaderState.java:329)
>   at 
> org.apache.ratis.server.impl.LeaderState.access$500(LeaderState.java:48)
> {code}
> This happens when followers is empty and not includeSelf == false.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-96) LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException

2017-07-31 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-96:


+1. I will commit the patch shortly.

> LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException
> -
>
> Key: RATIS-96
> URL: https://issues.apache.org/jira/browse/RATIS-96
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r96_20170720.patch, r96_20170725.patch, 
> r96_20170727.patch, r96_20170731.patch
>
>
> {code}
> java.lang.ArrayIndexOutOfBoundsException: 0
>   at 
> org.apache.ratis.server.impl.LeaderState.computeLastCommitted(LeaderState.java:490)
>   at 
> org.apache.ratis.server.impl.LeaderState.updateLastCommitted(LeaderState.java:400)
>   at 
> org.apache.ratis.server.impl.LeaderState.handleEvent(LeaderState.java:329)
>   at 
> org.apache.ratis.server.impl.LeaderState.access$500(LeaderState.java:48)
> {code}
> This happens when followers is empty and not includeSelf == false.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Resolved] (RATIS-99) Exclude shaded sources from the checkstyle check

2017-07-28 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-99?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-99.

   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

I've committed the patch. Thanks for the contribution, [~elek].

> Exclude shaded sources from the checkstyle check
> 
>
> Key: RATIS-99
> URL: https://issues.apache.org/jira/browse/RATIS-99
> Project: Ratis
>  Issue Type: Improvement
>Affects Versions: 0.2.0-alpha
>Reporter: Elek, Marton
>Assignee: Elek, Marton
> Fix For: 0.2.0-alpha
>
> Attachments: RATIS-99.001.patch
>
>
> The current checkstyle report is too noisy as all of the copied sources are 
> also checked. 
> I propose to exclude all the shaded files from proto-shaded/hadoop-shaded.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-97) Leader may not have majority after setConfiguration

2017-07-28 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-97:


bq. Currently, we just start new servers with one configuration.

I guess this is the issue. We must let the new server understand it's new 
member so we should pass null configuration for these peers. Then these peers 
will start from {{startInitializing}}.

I will check your test case as well later.

> Leader may not have majority after setConfiguration
> ---
>
> Key: RATIS-97
> URL: https://issues.apache.org/jira/browse/RATIS-97
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>
> Here is an example: Suppose a group has 3 servers and one of them is the 
> leader.  Use setConfiguration to add 6 more servers.  The leader is still 
> running as a leader although it does not has majority anymore.  The 6 new 
> servers may elect a new leader among them.  Finally, there are two leaders in 
> the group.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-97) Leader may not have majority after setConfiguration

2017-07-27 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-97:


Not sure whether this can happen. According to the protocol, during 
setConfiguration, a new leader needs to get majority from both the old 
configuration and the new configuration, thus we should be able to prevent the 
scenario where a new leader is directly selected among the 6 new servers. Do 
you have a test case for this scenario, Nicholas? If so we may have a bug for 
initializing new server.

> Leader may not have majority after setConfiguration
> ---
>
> Key: RATIS-97
> URL: https://issues.apache.org/jira/browse/RATIS-97
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>
> Here is an example: Suppose a group has 3 servers and one of them is the 
> leader.  Use setConfiguration to add 6 more servers.  The leader is still 
> running as a leader although it does not has majority anymore.  The 6 new 
> servers may elect a new leader among them.  Finally, there are two leaders in 
> the group.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-96) LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException

2017-07-27 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-96:


Uploaded the wrong patch? The latest patch is for ozone.

> LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException
> -
>
> Key: RATIS-96
> URL: https://issues.apache.org/jira/browse/RATIS-96
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r96_20170720.patch, r96_20170725.patch, 
> r96_20170727.patch
>
>
> {code}
> java.lang.ArrayIndexOutOfBoundsException: 0
>   at 
> org.apache.ratis.server.impl.LeaderState.computeLastCommitted(LeaderState.java:490)
>   at 
> org.apache.ratis.server.impl.LeaderState.updateLastCommitted(LeaderState.java:400)
>   at 
> org.apache.ratis.server.impl.LeaderState.handleEvent(LeaderState.java:329)
>   at 
> org.apache.ratis.server.impl.LeaderState.access$500(LeaderState.java:48)
> {code}
> This happens when followers is empty and not includeSelf == false.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-96) LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException

2017-07-20 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-96:


Thanks for fixing the issue, Nicholas! So is it possible that we can do an 
explicit check for "followers is empty and includeSelf is false" before calling 
{{computeLastCommitted}}? Then we do not need to use null/non-null Long to 
indicate this special case. Besides, maybe we can add a unit test for this?

> LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException
> -
>
> Key: RATIS-96
> URL: https://issues.apache.org/jira/browse/RATIS-96
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r96_20170720.patch
>
>
> {code}
> java.lang.ArrayIndexOutOfBoundsException: 0
>   at 
> org.apache.ratis.server.impl.LeaderState.computeLastCommitted(LeaderState.java:490)
>   at 
> org.apache.ratis.server.impl.LeaderState.updateLastCommitted(LeaderState.java:400)
>   at 
> org.apache.ratis.server.impl.LeaderState.handleEvent(LeaderState.java:329)
>   at 
> org.apache.ratis.server.impl.LeaderState.access$500(LeaderState.java:48)
> {code}
> This happens when followers is empty and not includeSelf == false.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Assigned] (RATIS-96) LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException

2017-07-20 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-96?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao reassigned RATIS-96:
--

Assignee: Tsz Wo Nicholas Sze

> LeaderState computeLastCommitted may throw ArrayIndexOutOfBoundsException
> -
>
> Key: RATIS-96
> URL: https://issues.apache.org/jira/browse/RATIS-96
> Project: Ratis
>  Issue Type: Bug
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r96_20170720.patch
>
>
> {code}
> java.lang.ArrayIndexOutOfBoundsException: 0
>   at 
> org.apache.ratis.server.impl.LeaderState.computeLastCommitted(LeaderState.java:490)
>   at 
> org.apache.ratis.server.impl.LeaderState.updateLastCommitted(LeaderState.java:400)
>   at 
> org.apache.ratis.server.impl.LeaderState.handleEvent(LeaderState.java:329)
>   at 
> org.apache.ratis.server.impl.LeaderState.access$500(LeaderState.java:48)
> {code}
> This happens when followers is empty and not includeSelf == false.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (RATIS-93) Filter peers from the iteration before random selection

2017-07-11 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-93:


+1

> Filter peers from the iteration before random selection
> ---
>
> Key: RATIS-93
> URL: https://issues.apache.org/jira/browse/RATIS-93
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Minor
> Attachments: r93_20170710.patch
>
>
> When randomly selecting a peer, we first copy the peer list and then select a 
> peer in a loop indefinitely until a new peer is found.
> We could first filter the peers when copying the list.  Then, we can 
> eliminate the loop.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Resolved] (RATIS-86) Support server re-initialization

2017-05-19 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-86?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-86.

   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

> Support server re-initialization
> 
>
> Key: RATIS-86
> URL: https://issues.apache.org/jira/browse/RATIS-86
> Project: Ratis
>  Issue Type: New Feature
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 0.2.0-alpha
>
> Attachments: r86_20170511.patch, r86_20170512b.patch, 
> r86_20170512.patch, r86_20170515.patch, r86_20170516.patch, r86_20170518.patch
>
>
> When a RaftServer is built, it creates a RaftServerImpl object and starts 
> running a RaftServerRpc service.  The RaftServerImpl captures ServerState 
> including RaftLog and RaftConfiguration.
> We want to support server re-initialization so that a server can support 
> multiple raft clusters sequentially.  This is our first step to support 
> multi-raft.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (RATIS-86) Support server re-initialization

2017-05-19 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-86:


+1. I will commit the patch shortly. Thanks Nicholas!

> Support server re-initialization
> 
>
> Key: RATIS-86
> URL: https://issues.apache.org/jira/browse/RATIS-86
> Project: Ratis
>  Issue Type: New Feature
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r86_20170511.patch, r86_20170512b.patch, 
> r86_20170512.patch, r86_20170515.patch, r86_20170516.patch, r86_20170518.patch
>
>
> When a RaftServer is built, it creates a RaftServerImpl object and starts 
> running a RaftServerRpc service.  The RaftServerImpl captures ServerState 
> including RaftLog and RaftConfiguration.
> We want to support server re-initialization so that a server can support 
> multiple raft clusters sequentially.  This is our first step to support 
> multi-raft.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-85) TestNotLeaderExceptionWithHadoopRpc and TestRaftReconfigurationWithHadoopRpc fail intermittently

2017-05-12 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-85?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-85:
---
Attachment: RATIS-85.000.patch

Upload the first patch.

> TestNotLeaderExceptionWithHadoopRpc and TestRaftReconfigurationWithHadoopRpc 
> fail intermittently
> 
>
> Key: RATIS-85
> URL: https://issues.apache.org/jira/browse/RATIS-85
> Project: Ratis
>  Issue Type: Bug
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-85.000.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (RATIS-87) Separate RaftServerImpl into proxy and impl

2017-05-10 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-87:


+1

> Separate RaftServerImpl into proxy and impl
> ---
>
> Key: RATIS-87
> URL: https://issues.apache.org/jira/browse/RATIS-87
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r87_20170505.patch, r87_20170508.patch, 
> r87_20170510.patch
>
>
> Separate RaftServerImpl into proxy and impl
> - Proxy: reads rpc requests and propagates them to impl.  It also can 
> start/stop  Impl.
> - Impl: captures the server state and handles rpc request.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Resolved] (RATIS-82) Add cache eviction policy

2017-05-10 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-82?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-82.

   Resolution: Fixed
Fix Version/s: 0.2.0-alpha

I've committed the patch. Thanks Nicholas for the review!

> Add cache eviction policy
> -
>
> Key: RATIS-82
> URL: https://issues.apache.org/jira/browse/RATIS-82
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Fix For: 0.2.0-alpha
>
> Attachments: RATIS-82.000.patch, RATIS-82.001.patch, 
> RATIS-82.002.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (RATIS-11) Better retry policy support for Raft client

2017-05-05 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-11:


[~jingc], sure! I've added you as Ratis contributor. 

> Better retry policy support for Raft client
> ---
>
> Key: RATIS-11
> URL: https://issues.apache.org/jira/browse/RATIS-11
> Project: Ratis
>  Issue Type: Improvement
>Reporter: Jing Zhao
>
> We should have better retry policy support on the raft client side, such as 
> different retry policies, more flexible retry parameter settings, etc.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-82) Add cache eviction policy

2017-05-04 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-82?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-82:
---
Attachment: RATIS-82.001.patch

Added some basic tests.

> Add cache eviction policy
> -
>
> Key: RATIS-82
> URL: https://issues.apache.org/jira/browse/RATIS-82
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-82.000.patch, RATIS-82.001.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-85) TestNotLeaderExceptionWithHadoopRpc and TestRaftReconfigurationWithHadoopRpc fail intermittently

2017-05-03 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-85?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-85:
---
Summary: TestNotLeaderExceptionWithHadoopRpc and 
TestRaftReconfigurationWithHadoopRpc fail intermittently  (was: 
TestNotLeaderExceptionWithHadoopRpc and TestRaftReconfigurationWithHadoopRpc 
fail immediately)

> TestNotLeaderExceptionWithHadoopRpc and TestRaftReconfigurationWithHadoopRpc 
> fail intermittently
> 
>
> Key: RATIS-85
> URL: https://issues.apache.org/jira/browse/RATIS-85
> Project: Ratis
>  Issue Type: Bug
>Reporter: Jing Zhao
>Assignee: Jing Zhao
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (RATIS-85) TestNotLeaderExceptionWithHadoopRpc and TestRaftReconfigurationWithHadoopRpc fail immediately

2017-05-03 Thread Jing Zhao (JIRA)
Jing Zhao created RATIS-85:
--

 Summary: TestNotLeaderExceptionWithHadoopRpc and 
TestRaftReconfigurationWithHadoopRpc fail immediately
 Key: RATIS-85
 URL: https://issues.apache.org/jira/browse/RATIS-85
 Project: Ratis
  Issue Type: Bug
Reporter: Jing Zhao
Assignee: Jing Zhao






--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (RATIS-65) Add a script to automate the release process

2017-04-28 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-65:


I've tested the script in my local environment and it works. +1

> Add a script to automate the release process
> 
>
> Key: RATIS-65
> URL: https://issues.apache.org/jira/browse/RATIS-65
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r65_20170427.patch
>
>
> As suggested by [~enis] below, we should add a script to automate the release 
> process.
> {quote}
> On an unrelated note, what can help us for driving frequent releases is to 
> have a script to drive the release process. I think we should do something 
> like this: 
> https://github.com/apache/hbase/blob/master/dev-support/make_rc.sh. 
> {quote}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (RATIS-63) Add NOTICE.txt

2017-04-28 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-63:


+1

> Add NOTICE.txt
> --
>
> Key: RATIS-63
> URL: https://issues.apache.org/jira/browse/RATIS-63
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: r63_20170412.patch
>
>
> According to [Licensing 
> Documentation|http://www.apache.org/legal/release-policy.html#licensing-documentation],
>  we need a NOTICE file.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (RATIS-82) Add cache eviction policy

2017-04-28 Thread Jing Zhao (JIRA)
Jing Zhao created RATIS-82:
--

 Summary: Add cache eviction policy
 Key: RATIS-82
 URL: https://issues.apache.org/jira/browse/RATIS-82
 Project: Ratis
  Issue Type: Sub-task
Reporter: Jing Zhao
Assignee: Jing Zhao






--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Resolved] (RATIS-76) Add loading policy for RaftLogCache

2017-04-26 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-76?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-76.

   Resolution: Fixed
Fix Version/s: 0.1.0-alpha

Thanks again for the review, Nicholas! I've committed the patch.

> Add loading policy for RaftLogCache
> ---
>
> Key: RATIS-76
> URL: https://issues.apache.org/jira/browse/RATIS-76
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Fix For: 0.1.0-alpha
>
> Attachments: RATIS-76.000.patch, RATIS-76.001.patch, 
> RATIS-76.002.patch, RATIS-76.003.patch
>
>
> RATIS-70 separates term/index/offset and entry content in {{LogSegment}}. Now 
> {{LogSegment}} can always holds term/index/offset part in memory as index, 
> and load log entry into a cache only when necessary. In this jira we will add 
> a cache loading policy in {{LogSegment}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (RATIS-76) Add loading policy for RaftLogCache

2017-04-25 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-76:


bq. Question: Is getEntryWithLoading intended to be only used in 
TestRaftLogSegment?

Originally I plan to use it in SegmentedRaftLog. Now looks like it is no longer 
useful. So I removed it from the 003 patch.

> Add loading policy for RaftLogCache
> ---
>
> Key: RATIS-76
> URL: https://issues.apache.org/jira/browse/RATIS-76
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-76.000.patch, RATIS-76.001.patch, 
> RATIS-76.002.patch, RATIS-76.003.patch
>
>
> RATIS-70 separates term/index/offset and entry content in {{LogSegment}}. Now 
> {{LogSegment}} can always holds term/index/offset part in memory as index, 
> and load log entry into a cache only when necessary. In this jira we will add 
> a cache loading policy in {{LogSegment}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-76) Add loading policy for RaftLogCache

2017-04-25 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-76?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-76:
---
Attachment: RATIS-76.003.patch

Thanks a lot for the review, Nicholas! Update the patch to address your 
comments.

> Add loading policy for RaftLogCache
> ---
>
> Key: RATIS-76
> URL: https://issues.apache.org/jira/browse/RATIS-76
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-76.000.patch, RATIS-76.001.patch, 
> RATIS-76.002.patch, RATIS-76.003.patch
>
>
> RATIS-70 separates term/index/offset and entry content in {{LogSegment}}. Now 
> {{LogSegment}} can always holds term/index/offset part in memory as index, 
> and load log entry into a cache only when necessary. In this jira we will add 
> a cache loading policy in {{LogSegment}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-78) Ratis does not compile in Intellij

2017-04-21 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-78?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-78:
---
Attachment: RATIS-78.000.patch

> Ratis does not compile in Intellij
> --
>
> Key: RATIS-78
> URL: https://issues.apache.org/jira/browse/RATIS-78
> Project: Ratis
>  Issue Type: Bug
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-78.000.patch
>
>
> Currently the ratis code base cannot get compiled in Intellij because the 
> Forked Tomcat Native cannot get located.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (RATIS-78) Ratis does not compile in Intellij

2017-04-21 Thread Jing Zhao (JIRA)
Jing Zhao created RATIS-78:
--

 Summary: Ratis does not compile in Intellij
 Key: RATIS-78
 URL: https://issues.apache.org/jira/browse/RATIS-78
 Project: Ratis
  Issue Type: Bug
Reporter: Jing Zhao
Assignee: Jing Zhao


Currently the ratis code base cannot get compiled in Intellij because the 
Forked Tomcat Native cannot get located.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-76) Add loading policy for RaftLogCache

2017-04-21 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-76?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-76:
---
Attachment: RATIS-76.002.patch

Update the patch with some simple tests. Also let the SegmentedRaftLog call 
cache.load explicitly so that we can avoid disk IO within the 
SegmentedRaftLog's lock.

> Add loading policy for RaftLogCache
> ---
>
> Key: RATIS-76
> URL: https://issues.apache.org/jira/browse/RATIS-76
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-76.000.patch, RATIS-76.001.patch, 
> RATIS-76.002.patch
>
>
> RATIS-70 separates term/index/offset and entry content in {{LogSegment}}. Now 
> {{LogSegment}} can always holds term/index/offset part in memory as index, 
> and load log entry into a cache only when necessary. In this jira we will add 
> a cache loading policy in {{LogSegment}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (RATIS-77) "RaftServerProtocolService" and " GRpc.proto" inconsistent

2017-04-21 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-77:


Thank you for reporting the issue, [~kaiyangzhang]! Yeah, looks like we should 
change the protobuf definition to make it consistent. Do you want to upload a 
patch to fix?

> "RaftServerProtocolService" and " GRpc.proto" inconsistent
> --
>
> Key: RATIS-77
> URL: https://issues.apache.org/jira/browse/RATIS-77
> Project: Ratis
>  Issue Type: Bug
>Reporter: kaiyangzhang
>Assignee: Jing Zhao
>
> *Maybe a bug, look at the comments in the code and GRpc.proto*
> {code:title=GRpc.proto|borderStyle=solid}
>..
>..
>//Executes a client-streaming call , return only one response.
>rpc installSnapshot(stream ratis.common.InstallSnapshotRequestProto)
>   returns(ratis.common.InstallSnapshotReplyProto) {}
>...
>...
> {code}
> {code:title=RaftServerProtocolService.java |borderStyle=solid}
> 
> 
> @Override
>   public StreamObserver installSnapshot(
>   StreamObserver responseObserver) {
> return new StreamObserver() {
>   @Override
>   public void onNext(InstallSnapshotRequestProto request) {
> try {
>   //receive a client-streaming call, return a response stream
>   //code and GRpc.proto inconsistent
>   final InstallSnapshotReplyProto reply = 
> server.installSnapshot(request);
>   responseObserver.onNext(reply);
> } catch (Throwable e) {
>   LOG.info("{} got exception when handling installSnapshot {}: {}",
>   id, request.getServerRequest(), e);
>   responseObserver.onError(RaftGrpcUtil.wrapException(e));
> }
>   }
>..
>..
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-76) Add loading policy for RaftLogCache

2017-04-20 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-76?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-76:
---
Attachment: RATIS-76.001.patch

Added RaftLogIOException for Exception thrown while loading log entries from 
log files.

> Add loading policy for RaftLogCache
> ---
>
> Key: RATIS-76
> URL: https://issues.apache.org/jira/browse/RATIS-76
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-76.000.patch, RATIS-76.001.patch
>
>
> RATIS-70 separates term/index/offset and entry content in {{LogSegment}}. Now 
> {{LogSegment}} can always holds term/index/offset part in memory as index, 
> and load log entry into a cache only when necessary. In this jira we will add 
> a cache loading policy in {{LogSegment}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Resolved] (RATIS-70) Separate term/index/offset and log entry content in LogSegment

2017-04-17 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-70?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-70.

   Resolution: Fixed
Fix Version/s: 0.1.0-alpha

Thanks Nicholas for the review! I've committed the patch.

> Separate term/index/offset and log entry content in LogSegment
> --
>
> Key: RATIS-70
> URL: https://issues.apache.org/jira/browse/RATIS-70
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Fix For: 0.1.0-alpha
>
> Attachments: RATIS-70.000.patch, RATIS-70.001.patch, 
> RATIS-70.002.patch
>
>
> The current RaftLogCache consists of LogSegment, and logSegment consists of 
> log entries. Instead of directly storing all the log entries, we should 
> separate the term/index/offset information and the entry content information. 
> The former part is more like the index information and can be always kept in 
> the memory. The entry content part can later be evicted from the memory based 
> on eviction policies.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-76) Add loading policy for RaftLogCache

2017-04-13 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-76?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-76:
---
Attachment: RATIS-76.000.patch

Initial patch to demo the idea. The patch depends on RATIS-70. Will add more 
tests later.

> Add loading policy for RaftLogCache
> ---
>
> Key: RATIS-76
> URL: https://issues.apache.org/jira/browse/RATIS-76
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-76.000.patch
>
>
> RATIS-70 separates term/index/offset and entry content in {{LogSegment}}. Now 
> {{LogSegment}} can always holds term/index/offset part in memory as index, 
> and load log entry into a cache only when necessary. In this jira we will add 
> a cache loading policy in {{LogSegment}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (RATIS-76) Add loading policy for RaftLogCache

2017-04-13 Thread Jing Zhao (JIRA)
Jing Zhao created RATIS-76:
--

 Summary: Add loading policy for RaftLogCache
 Key: RATIS-76
 URL: https://issues.apache.org/jira/browse/RATIS-76
 Project: Ratis
  Issue Type: Sub-task
Reporter: Jing Zhao
Assignee: Jing Zhao


RATIS-70 separates term/index/offset and entry content in {{LogSegment}}. Now 
{{LogSegment}} can always holds term/index/offset part in memory as index, and 
load log entry into a cache only when necessary. In this jira we will add a 
cache loading policy in {{LogSegment}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-70) Separate term/index/offset and log entry content in LogSegment

2017-04-13 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-70?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-70:
---
Attachment: RATIS-70.002.patch

Thanks a lot for the review, Nicholas! Update the patch to address your 
comments. Also added RaftLog#getTermIndex(long) and used it instead of 
RaftLog#get(long) in case that we only need TermIndex information.

bq. It look like that LogSegment.configEntries is not needed since the entry is 
in entryCache

The main usage of configEntries is to remove the dependency of the entry cache 
when checking if an entry with the given index is for re-configuration. The 
current code still has all the entries in the cache, but later we will add 
eviction/loading policies thus the assumption will no longer hold. 

> Separate term/index/offset and log entry content in LogSegment
> --
>
> Key: RATIS-70
> URL: https://issues.apache.org/jira/browse/RATIS-70
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-70.000.patch, RATIS-70.001.patch, 
> RATIS-70.002.patch
>
>
> The current RaftLogCache consists of LogSegment, and logSegment consists of 
> log entries. Instead of directly storing all the log entries, we should 
> separate the term/index/offset information and the entry content information. 
> The former part is more like the index information and can be always kept in 
> the memory. The entry content part can later be evicted from the memory based 
> on eviction policies.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (RATIS-75) Document Ratis library usage

2017-04-13 Thread Jing Zhao (JIRA)
Jing Zhao created RATIS-75:
--

 Summary: Document Ratis library usage
 Key: RATIS-75
 URL: https://issues.apache.org/jira/browse/RATIS-75
 Project: Ratis
  Issue Type: Improvement
Reporter: Jing Zhao
Priority: Critical


We need to provide document about how to use Ratis library. Things to cover:
# How to run ratis on top of your current RPC engine
# How to integrate your state machine



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Assigned] (RATIS-74) Missing dependency for enforcer rule enforceBytecodeVersion

2017-04-13 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-74?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao reassigned RATIS-74:
--

Assignee: Karl Heinz Marbaise

> Missing dependency for enforcer rule enforceBytecodeVersion
> ---
>
> Key: RATIS-74
> URL: https://issues.apache.org/jira/browse/RATIS-74
> Project: Ratis
>  Issue Type: Bug
>Affects Versions: 0.1.0-alpha
>Reporter: Karl Heinz Marbaise
>Assignee: Karl Heinz Marbaise
>Priority: Critical
> Fix For: 0.1.0-alpha
>
> Attachments: 
> 0001-RATIS-74-Missing-dependency-for-enforcer-rule-enforc.patch
>
>
> If you using the profile {{release}} you will get a failure based on the 
> missing dependency for the {{enforceBytecodeVersion}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Assigned] (RATIS-73) Update apache-rat-plugin to version 0.12

2017-04-13 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-73?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao reassigned RATIS-73:
--

Assignee: Karl Heinz Marbaise

> Update apache-rat-plugin to version 0.12
> 
>
> Key: RATIS-73
> URL: https://issues.apache.org/jira/browse/RATIS-73
> Project: Ratis
>  Issue Type: Improvement
>Affects Versions: 0.1.0-alpha
>Reporter: Karl Heinz Marbaise
>Assignee: Karl Heinz Marbaise
>Priority: Trivial
> Fix For: 0.1.0-alpha
>
> Attachments: 
> 0001-RATIS-73-Update-apache-rat-plugin-to-version-0.12.patch
>
>
> Upgrading the apache-rat-plugin to 0.12 and remove the rest of the 
> configuration cause it's the default.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-70) Separate term/index/offset and log entry content in LogSegment

2017-04-11 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-70?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-70:
---
Attachment: RATIS-70.000.patch

Rebase the patch.

> Separate term/index/offset and log entry content in LogSegment
> --
>
> Key: RATIS-70
> URL: https://issues.apache.org/jira/browse/RATIS-70
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Attachments: RATIS-70.000.patch
>
>
> The current RaftLogCache consists of LogSegment, and logSegment consists of 
> log entries. Instead of directly storing all the log entries, we should 
> separate the term/index/offset information and the entry content information. 
> The former part is more like the index information and can be always kept in 
> the memory. The entry content part can later be evicted from the memory based 
> on eviction policies.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (RATIS-70) Separate term/index/offset and log entry content in LogSegment

2017-04-06 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-70?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao updated RATIS-70:
---
Summary: Separate term/index/offset and log entry content in LogSegment  
(was: Code refactoring on RaftLogCache)

> Separate term/index/offset and log entry content in LogSegment
> --
>
> Key: RATIS-70
> URL: https://issues.apache.org/jira/browse/RATIS-70
> Project: Ratis
>  Issue Type: Sub-task
>Reporter: Jing Zhao
>Assignee: Jing Zhao
>
> The current RaftLogCache consists of LogSegment, and logSegment consists of 
> log entries. Instead of directly storing all the log entries, we should 
> separate the term/index/offset information and the entry content information. 
> The former part is more like the index information and can be always kept in 
> the memory. The entry content part can later be evicted from the memory based 
> on eviction policies.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Resolved] (RATIS-68) Simplify parent entries

2017-04-06 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-68?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-68.

Resolution: Fixed

+1. Thanks for the fix, [~khmarbaise]!

> Simplify parent entries
> ---
>
> Key: RATIS-68
> URL: https://issues.apache.org/jira/browse/RATIS-68
> Project: Ratis
>  Issue Type: Improvement
>Affects Versions: 0.1.0-alpha
>Reporter: Karl Heinz Marbaise
>Assignee: Karl Heinz Marbaise
>Priority: Trivial
> Fix For: 0.1.0-alpha
>
> Attachments: 0001-RATIS-68-Simplify-parent-entries.patch
>
>
> Every child pom has in it's parent:
> {code:xml}
>   4.0.0
>   
> ratis
> org.apache.ratis
> 0.1-SNAPSHOT
> ..
>   
>   ..
>   0.1-SNAPSHOT
> {code} where the entry for {{relativePath}} is simply not needed, cause it 
> the default in Maven.
> Furthermore each child defines the version which is also not needed cause 
> it's inherited from the parent. This can be simplified.
> I have attached an appropriate patch to fix the issue.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (RATIS-70) Code refactoring on RaftLogCache

2017-04-06 Thread Jing Zhao (JIRA)
Jing Zhao created RATIS-70:
--

 Summary: Code refactoring on RaftLogCache
 Key: RATIS-70
 URL: https://issues.apache.org/jira/browse/RATIS-70
 Project: Ratis
  Issue Type: Sub-task
Reporter: Jing Zhao
Assignee: Jing Zhao


In this jira we do some code refactoring on RaftLogCache. More specifically, we 
will let RaftLogCache only expose interfaces about log entries.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (RATIS-64) Do not pre-shade artifacts

2017-04-03 Thread Jing Zhao (JIRA)

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

Jing Zhao commented on RATIS-64:


Yes maybe we can use this chance to remove hadoop related code from the ratis 
library. I agree with Enis that the ratis-hadoop code belongs more to hadoop 
and we can reuse it when we use ratis in Hadoop (e.g. NameNode HA).

> Do not pre-shade artifacts
> --
>
> Key: RATIS-64
> URL: https://issues.apache.org/jira/browse/RATIS-64
> Project: Ratis
>  Issue Type: Bug
>Reporter: Enis Soztutar
>
> Hugo and I were discussing a case were the shading should happen at the last 
> step, rather than what we do today. 
> I think there are 3 possible strategies of shading that one can do: 
>  (1) pre-shade some of your dependencies, so that your other dependencies can 
> work. This what we do today, we shade PB+ GRPC, etc so that Hadoop can work. 
>  (2) pre-shade some of your dependencies' transitive dependencies so that you 
> depend on already-shaded artifacts. This will be like having maven artifacts 
> of shaded-hadoop so that hadoop itself does not bring in any more dependency. 
> If hadoop has shaded artifacts, or we do shading of hadoop's dependencies in 
> another repository, we won't need to pre-shade PB, etc. 
>  (3) post-shade. This means that in the code itself we do not depend on 
> shaded packages anymore, but do the shading as a different module so that we 
> publish shaded artifacts. This allows the downstreamers to be able to consume 
> ratis, while allowing ratis source code to be saner. 
> Obviously for doing (3), we need to kick out ratis-hadoop module to the 
> hadoop project. Thinking about it, I think ratis-hadoop does not belong in 
> Ratis itself anyway. What do you guys think about moving the code over to 
> Hadoop, and getting rid of the hadoop dependency to end this shading madness? 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Resolved] (RATIS-62) Return the Exception from StateMachine#preAppendTransaction to client as StateMachineException

2017-03-31 Thread Jing Zhao (JIRA)

 [ 
https://issues.apache.org/jira/browse/RATIS-62?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jing Zhao resolved RATIS-62.

   Resolution: Fixed
Fix Version/s: 0.1.0-alpha

Thanks for the review, Mingliang! I've committed the patch.

> Return the Exception from StateMachine#preAppendTransaction to client as 
> StateMachineException
> --
>
> Key: RATIS-62
> URL: https://issues.apache.org/jira/browse/RATIS-62
> Project: Ratis
>  Issue Type: Bug
>Reporter: Jing Zhao
>Assignee: Jing Zhao
> Fix For: 0.1.0-alpha
>
> Attachments: RATIS-62.000.patch
>
>
> This is a TODO in RaftServerImpl#appendTransaction. The IOException thrown by 
> {{applyLog}} is actually from the {{StateMachine#preAppendTransaction}}. This 
> exception should be return to RaftClient as a {{StateMachineException}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


  1   2   >