[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-18 Thread Josh Elser (JIRA)


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

Josh Elser commented on RATIS-274:
--

Applies cleanly on 940a169bac12b10eb636d62029dddbc9fad420ff. Easy rebase on top 
of RATIS-354.

Let me look at this – would like to merge it today.

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch, RATIS-274-v2.patch, 
> RATIS-274-v3.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-17 Thread Josh Elser (JIRA)


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

Josh Elser commented on RATIS-274:
--

[~vrodionov], maybe needs a rebase? v3 doesn't seem to apply to HEAD (which is 
9774c5cb10dc14b280c8f5bafd852c0581ca0845 for me)

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch, RATIS-274-v2.patch, 
> RATIS-274-v3.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-17 Thread Vladimir Rodionov (JIRA)


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

Vladimir Rodionov commented on RATIS-274:
-

Patch v3. Fixed [~rajeshbabu] end-to-end test. 

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch, RATIS-274-v2.patch, 
> RATIS-274-v3.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-16 Thread Ted Yu (JIRA)


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

Ted Yu commented on RATIS-274:
--

{code}
+  public ByteBuffer readNext() throws IOException {
+int num = 1;
+RaftClientReply reply =
+raftClient.sendReadOnly(Message.valueOf(LogServiceProtoUtil
+.toReadLogRequestProto(parent.getName(), currentRecordId, num)
{code}
It seems num is not modified within the method. It can be hardcoded directly.
{code}
+proto.getLogRecord(0);
{code}
The above line can be removed.
{code}
+if (proto.hasException()) {
+  LogServiceException e = proto.getException();
+  throw new IOException(e.getErrorMsg());
{code}
It would be better to wrap IOE around the exception, e, so that we have more 
clue as for the cause.

It seems the body of the two {{readNext}} methods can be unified - they look 
mostly the same.
{code}
+currentRecordId+= numRecords;
+List ret = new ArrayList();
+int n = proto.getLogRecordCount();
{code}
Should we check n == numRecords ?
{code}
+  public Collection getRecordListeners() {
+return listeners;
{code}
I think returning unmodifiable collection would be better.
{code}
+  List listeners;
{code}
If order of listeners is not important, Set can be used.

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch, RATIS-274-v2.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-15 Thread Josh Elser (JIRA)


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

Josh Elser commented on RATIS-274:
--

{code:java}
+for (long index = startRecordId; index < startRecordId + num; index++) {
+  try {
+
list.add(log.getEntryWithData(index).getEntry().getSmLogEntry().getData().toByteArray());
+  } catch(RaftLogIOException e) {
+t = e;
+list = null;
+break;
+  }
+}{code}
Two things I'm curious if you've considered: first, what kind of exceptions 
might happen that would throw the RaftLogIOException and if this is something 
that would be recoverable/retriable? Second, can we accidentally read "off the 
end" of the log?
{code:java}
diff --git 
a/ratis-logservice/src/main/java/org/apache/ratis/logservice/dummy/DummyLogService.java
 
b/ratis-logservice/src/main/java/org/apache/ratis/logservice/dummy/DummyLogService.java{code}
If we've gone "past" the usefulness of these, please delete them by all means :)
{quote}Top level tests (with Raft client) are not working yet. I have to figure 
out why raft cluster does not start.
{quote}
Please shout if you need a hand!
{quote} What do you think if we separate meta and log operations?
{quote}
(even though I'm not being addressed :P) I like your suggestions, Sergey!

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch, RATIS-274-v2.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-15 Thread Vladimir Rodionov (JIRA)


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

Vladimir Rodionov commented on RATIS-274:
-

Patch v2. 

Top level tests (with Raft client) are not working yet. I have to figure out 
why raft cluster does not start. 

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch, RATIS-274-v2.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-15 Thread Sergey Soldatov (JIRA)


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

Sergey Soldatov commented on RATIS-274:
---

[~vrodionov]  What do you think if we separate meta and log operations? 
LogService class itself looks very confusing. By the name, it's supposed to be 
a service, but actually, that's a client (and it's not clear which client). 
I.e. to communicate to meta we need a single raft client. But for each log we 
need a separate raft client (different quorum peers, different raft groups).  
How about such end-to-end scenario:
{code}
LogServiceClient client = new LogServiceClient(metaQuorum);
LogStream stream = client.createLog(LogName.of("test"));
LogReader reader = stream.getReader();
LogWriter writer = stream.getWriter();
LogStatus status = stream.getStatus(); 
{code}
That would eliminate the confusing class LogService, no need for any kind of 
fabrics, etc. Internally I would suggest to introduce a class that has a pair 
 that we will keep inside of the LogStream, so no need for 
LogStream.getLogService().getRaftClient() calls and instead it will be able to 
create its own client when it's required. Or we may create that client even 
earlier (during createLog() call) and pass it in the constructor of the 
LogStream.  

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-06 Thread Ted Yu (JIRA)


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

Ted Yu commented on RATIS-274:
--

{code}
+  LogStream createLog(LogName name, LogServiceConfiguration config);
{code}
The single parameter createLog throws exception. Should the above method throw 
exception as well ?
{code}
+   * @return
*/
-  void removeRecordListener(LogName name, RecordListener listener);
+  boolean removeRecordListener(LogName name, RecordListener listener);
{code}
What does the return value represent ?

For LogServiceConfiguration#set, Map#put returns:
the previous value associated with key, or null if there was no mapping for key

It would be good to align the set with Map#put in terms of return value.
{code}
+try {
+  log.close();
+  close();
+} catch (IOException e) {
{code}
If log.close throws exception, wouldn't close() call be skipped ?



> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-05 Thread Josh Elser (JIRA)


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

Josh Elser commented on RATIS-274:
--

{quote}Yes, if you think that we can keep LogService.proto in logservice module 
, you can do this via separate JIRA, [~elserj]. It is independent from this 
patch.
{quote}
Fine by me. WIll create.
{quote}We can push RATIS-279 first (with proto module changes) first, then I 
will rebase my own patch, after that we can think about separating DDL and DML 
code. This is my plan. What do you think, guys? [~elserj], [~rajeshbabu]?
{quote}
If that works for you, that works for me! I just want to make sure you and 
Rajesh both have the ability to keep moving forward.

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-05 Thread Vladimir Rodionov (JIRA)


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

Vladimir Rodionov commented on RATIS-274:
-

... On RaftClient 
 I will try to make it package private, although it is a not that easy now 
(interface and impl are in different subpackages)

Yes, if you think that we can keep LogService.proto in logservice module , you 
can do this via separate JIRA, [~elserj]. It is independent from this patch.

We can push RATIS-279 first (with proto module changes) first, then I  will 
rebase my own patch, after that we can think about separating DDL and DML code. 
This is my plan. What do you think, guys? [~elserj], [~rajeshbabu]? 


> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-04 Thread Josh Elser (JIRA)


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

Josh Elser commented on RATIS-274:
--

{code:java}
+  /**
+   * Gets Raft client object
+   * @return raft client object
+   */
+  RaftClient getRaftClient();{code}
I'm hesitant about having RaftClient in our API. (rhetorical) Are we use that 
Ratis is going to keep this stable? Glancing at the rest of the patch, couldn't 
we hide this on the implementation?
{code:java}
+  private CompletableFuture processSyncRequest(TransactionContext trx,
+  SyncRequestMessage logMessage) {
+
+//TODO really need this?
+takeSnapshot();
+return CompletableFuture.completedFuture(new SyncReplyMessage(null));
+
+  }{code}
Yeah, this doesn't really make sense, does it? Every action should already be 
"synced" after it is visible in the statemachine. I was originally thinking 
about this in regards to the group-commit we'll want in HBase, but now I'm not 
so sure what it's giving us here.
{code:java}
--- /dev/null
+++ b/ratis-proto/src/main/proto/LogService.proto{code}
Any reason to not keep this in the ratis-logservice module? I can help with the 
pom changes if necessary.
{code:java}
+case CLOSE:
+  return processCloseRequest(trx, 
(CloseRequestMessage)logMessage);{code}
I gave the suggestion to Rajesh in RATIS-279 that we try to separate out these 
DDL operations from the DML operations via an interface, in hopes of keeping 
you and he from stepping on each other. Do you see a better way?

Mentioned this on RATIS-279 too: a visitor pattern to eliminate the switch 
statements for RPC dispatch would be nice.

Do you want to try to split this up and commit in smaller chunks or keep 
building, Vlad? Looks like you're on a good path.

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch
>
>
> Implement the ability to read/write data to a log stream.



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


[jira] [Commented] (RATIS-274) Read/Write-path of log stream state machine

2018-10-03 Thread Vladimir Rodionov (JIRA)


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

Vladimir Rodionov commented on RATIS-274:
-

First cut: skeleton + some muscles :)

> Read/Write-path of log stream state machine
> ---
>
> Key: RATIS-274
> URL: https://issues.apache.org/jira/browse/RATIS-274
> Project: Ratis
>  Issue Type: Sub-task
>  Components: LogService
>Reporter: Josh Elser
>Assignee: Vladimir Rodionov
>Priority: Major
> Attachments: RATIS-274-v1.patch
>
>
> Implement the ability to read/write data to a log stream.



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