[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-31 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r214448422
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogName.java ---
@@ -0,0 +1,48 @@
+package org.apache.ratis.logservice.api;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Objects;
+
+/**
+ * Identifier to uniquely identify a {@link LogStream}.
+ */
+public class LogName {
+  // It's pretty likely that what uniquely defines a LogStream
+  // to change over time. We should account for this by making an
+  // API which can naturally evolve.
+  private final String name;
+
+  private LogName(String name) {
+this.name = requireNonNull(name);
+  }
+
+  // Implementation detail -- we want uses to use the LogName as 
identifiable, not to
--- End diff --

Nice! Didn't know about these. Happy to add that.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-30 Thread ss77892
Github user ss77892 commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r214160302
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogService.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Entry point for interacting with the Ratis LogService.
+ */
+public interface LogService {
+
+  /**
+   * Creates a new {@link LogStream} identified by the given name. Throws
+   * an exception if a {@link LogStream} with the given name already 
exists.
+   *
+   * @param name Unique name for this LogStream.
+   */
+  CompletableFuture createLog(LogName name);
+
+  /**
+   * Fetches the {@link LogStream} identified by the given name.
+   *
+   * @param name The name of the LogStream
+   */
+  CompletableFuture getLog(LogName name);
+
+  /**
+   * Lists all {@link LogStream} instances known by this LogService.
+   */
+  CompletableFuture> listLogs();
--- End diff --

@VladRodionov I completely agree with you that it's possible to implement 
more than one log using a single state machine. I was talking about the current 
Ratis implementation which has a single log per state machine. The benefit we 
have is that the replication of this log is a part of the implementation and we 
shouldn't implement anything on our own. If I understand correctly you suggest 
implementing multiple logs separately. That would mean 1) double writes (still 
all that data will go to RatisLog)  2) Special handling for those records to 
write it to the correct log. From my point of view, the performance of the 
write path is more critical for us compared to the read path.  
Just in case there is a concept of 'multi-raft' already implemented by a 
number of projects. We may get it implemented on Ratis level, but I'm not sure 
how it's suitable in the embedded library concept and how much efforts it would 
require. 



---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-29 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r213797305
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStream.java 
---
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A distributed log with "infinite" length that supports reads and writes.
+ */
+public interface LogStream {
+
+  /**
+   * Returns the unique name to identify this log.
+   */
+  LogName getName();
+
+  /**
+   * Returns the size of this LogStream in bytes.
+   */
+  long getSizeInBytes();
+
+  /**
+   * Returns the number of records in this LogStream.
+   */
+  long getSizeInRecords();
+
+  /**
+   * Creates a reader to read this LogStream asynchronously.
+   *
+   * @return An asynchronous reader
+   */
+  AsyncLogReader createAsyncReader();
+
+  /**
+   * Creates a writer to write to this LogStream asynchronously.
+   *
+   * @return An asynchronous writer
+   */
+  AsyncLogWriter createAsyncWriter();
+
+  /**
+   * Creates a reader to read this LogStream synchronously.
+   *
+   * @return A synchronous reader
+   */
+  LogReader createReader();
+
+  /**
+   * Creates a write to write to this LogStream synchronously.
+   *
+   * @return A synchronous writer
+   */
+  LogWriter createWriter();
+
+  /**
+   * Removes the elements in this LogStream prior to the given recordId.
+   *
+   * @param recordId A non-negative recordId for this LogStream
+   */
+  CompletableFuture truncateBefore(long recordId);
+
+  /**
+   * Returns the recordId which is the start of the LogStream. When there 
are records which were truncated
+   * from the LogStream, this will return a value larger than {@code 0}.
+   */
+  CompletableFuture getFirstRecordId();
+
+  /**
+   * Copies all records from the beginning of the LogStream until the 
given {@code recordId}
+   * to the configured archival storage.
+   *
+   * @param recordId A non-negative recordId for this LogStream
+   */
+  CompletableFuture archiveBefore(long recordId);
+
+  /**
+   * Returns the recordId, prior to which, all records in the LogStream 
are archived.
+   */
+  CompletableFuture getArchivalPoint();
--- End diff --

Yeah, need to rework this based on the doc I updated on RATIS-271 -- thanks!


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-29 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r213797238
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/RecordListener.java
 ---
@@ -0,0 +1,19 @@
+package org.apache.ratis.logservice.api;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Interface that, when registered with a {@link LogStream}, will receive 
all records written
+ * to that LogStream until it is removed.
+ */
+public interface RecordListener {
+
+  /**
+   * Processes the written record from the LogStream.
+   *
+   * @param record The record
+   */
+  CompletableFuture receiveRecord(ByteBuffer record);
--- End diff --

Dropping this with the other async stuff.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-29 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r213797086
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogReader.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Synchronous client interface to read from a LogStream.
+ */
+public interface LogReader extends AutoCloseable {
+
+  /**
+   * Seeks to the position before the record at the provided {@code 
offset} in the LogStream.
+   *
+   * @param offset A non-negative, offset in the LogStream
+   * @return A future for when the operation is completed.
+   */
+  void seek(long recordId) throws IOException;
--- End diff --

Moving to a specific point in a LogStream is absolutely a necessary API 
call. We must be able to resume reading from a known point in the Log. HBase 
Replication needs that functionality.

A naive implementation can read all records from the LogStream to know 
where to start reading. A slightly more advanced implementation can use the 
underlying segment files to find the right segment file to read from. An 
advance implementation could consult an index which we explicitly maintain over 
all segment files.

We haven't touched on implementation of how the LogReader class would 
negotiate this call with the back-end StateMachine (that depends on what the 
StateMachine and/or Ratis APIs look like, I think).


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-29 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r213795159
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/ArchivedLog.java 
---
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+/**
+ * A {@link LogStream} which has been archived in some external
+ * system. This interface is parameterized to allow for implementations
+ * to use their own class to encapsulate how to find the archived log.
+ *
+ * In the majority of cases, this should be transparent to end-users, as
+ * the {@link LogStream} should hide the fact that this even exists.
+ * TODO maybe that means this should be client-facing at all?
+ *
+ * @param  A referent to the log on the external system.
+ */
+public interface ArchivedLog extends AutoCloseable {
+
+  /**
+   * Creates an asynchronous reader over this archived log.
+   */
+  AsyncLogReader createAsyncReader();
--- End diff --

Dropping all of the async API for now. Might help later but we should do 
that with purpose.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-23 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212458341
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStream.java 
---
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A distributed log with "infinite" length that supports reads and writes.
+ */
+public interface LogStream {
+
+  /**
+   * Returns the unique name to identify this log.
+   */
+  LogName getName();
+
+  /**
+   * Returns the size of this LogStream in bytes.
+   */
+  long getSizeInBytes();
+
+  /**
+   * Returns the number of records in this LogStream.
+   */
+  long getSizeInRecords();
+
+  /**
+   * Creates a reader to read this LogStream asynchronously.
+   *
+   * @return An asynchronous reader
+   */
+  AsyncLogReader createAsyncReader();
+
+  /**
+   * Creates a writer to write to this LogStream asynchronously.
+   *
+   * @return An asynchronous writer
+   */
+  AsyncLogWriter createAsyncWriter();
+
+  /**
+   * Creates a reader to read this LogStream synchronously.
+   *
+   * @return A synchronous reader
+   */
+  LogReader createReader();
+
+  /**
+   * Creates a write to write to this LogStream synchronously.
+   *
+   * @return A synchronous writer
+   */
+  LogWriter createWriter();
+
+  /**
+   * Removes the elements in this LogStream prior to the given recordId.
+   *
+   * @param recordId A non-negative recordId for this LogStream
+   */
+  CompletableFuture truncateBefore(long recordId);
--- End diff --

A LogStream is a sequence of entries:  and RecordId is 
monotonically increasing. Truncation is the removal of entries from the "head" 
of the stream.

e.g. if we have a log with recordIds from `[0,100]`, we might say 
`truncateBefore(10, false)` which would remove entries 0, 1, 2, ..., 9. The 
"head" of the log would then be `recordId=10`


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-23 Thread VladRodionov
Github user VladRodionov commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212385674
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogReader.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Synchronous client interface to read from a LogStream.
+ */
+public interface LogReader extends AutoCloseable {
+
+  /**
+   * Seeks to the position before the record at the provided {@code 
offset} in the LogStream.
+   *
+   * @param offset A non-negative, offset in the LogStream
+   * @return A future for when the operation is completed.
+   */
+  void seek(long recordId) throws IOException;
--- End diff --

I meant what I meant: logs are not a message queues. Period. There is no 
need for random access in log file to grab a particular log record, but there 
is such a need in message queue. As for seeking support in Raft log - there is 
no such requirement to implement this efficiently, quite contrary, afair, 
protocol describes very stupid and lazy algo of synching two logs: leader goes 
backward one record at a time and compares it with a lagging follower records. 
Do not bring please functionality we (Log service users) are not asking 
for.  


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-23 Thread ss77892
Github user ss77892 commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212202468
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogService.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Entry point for interacting with the Ratis LogService.
+ */
+public interface LogService {
+
+  /**
+   * Creates a new {@link LogStream} identified by the given name. Throws
+   * an exception if a {@link LogStream} with the given name already 
exists.
+   *
+   * @param name Unique name for this LogStream.
+   */
+  CompletableFuture createLog(LogName name);
+
+  /**
+   * Fetches the {@link LogStream} identified by the given name.
+   *
+   * @param name The name of the LogStream
+   */
+  CompletableFuture getLog(LogName name);
+
+  /**
+   * Lists all {@link LogStream} instances known by this LogService.
+   */
+  CompletableFuture> listLogs();
--- End diff --

I would better ask whether this method should be here. Ratis is an embedded 
library that allows creating a SINGLE distributed state machine. That means a 
SINGLE log. A distributed log service that would allow creating a number of 
logs (i.e. state machines) sounds like an application that would include state 
machines creation/management. That still can be a part of the embedded library, 
but it would make everything more complicated and less flexible (i.e. different 
use cases may require special rules for quorum members like rack awareness, 
latency measurement, RS Groups in case of HBase, etc). 


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-23 Thread ss77892
Github user ss77892 commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212197492
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogReader.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Synchronous client interface to read from a LogStream.
+ */
+public interface LogReader extends AutoCloseable {
+
+  /**
+   * Seeks to the position before the record at the provided {@code 
offset} in the LogStream.
+   *
+   * @param offset A non-negative, offset in the LogStream
+   * @return A future for when the operation is completed.
+   */
+  void seek(long recordId) throws IOException;
--- End diff --

That's why most of the implementations of distributed logs (pub/sub model) 
actually are using segments/slices/whatever they call it and why usually they 
are relatively small. Usually, there is a separate index that maintains the map 
between id and segment OR it keeps start id for the segment in its name.  What 
do you mean by 'separate logs from message queues'? 


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-22 Thread VladRodionov
Github user VladRodionov commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212154446
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStream.java 
---
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A distributed log with "infinite" length that supports reads and writes.
+ */
+public interface LogStream {
+
+  /**
+   * Returns the unique name to identify this log.
+   */
+  LogName getName();
+
+  /**
+   * Returns the size of this LogStream in bytes.
+   */
+  long getSizeInBytes();
+
+  /**
+   * Returns the number of records in this LogStream.
+   */
+  long getSizeInRecords();
+
+  /**
+   * Creates a reader to read this LogStream asynchronously.
+   *
+   * @return An asynchronous reader
+   */
+  AsyncLogReader createAsyncReader();
+
+  /**
+   * Creates a writer to write to this LogStream asynchronously.
+   *
+   * @return An asynchronous writer
+   */
+  AsyncLogWriter createAsyncWriter();
+
+  /**
+   * Creates a reader to read this LogStream synchronously.
+   *
+   * @return A synchronous reader
+   */
+  LogReader createReader();
+
+  /**
+   * Creates a write to write to this LogStream synchronously.
+   *
+   * @return A synchronous writer
+   */
+  LogWriter createWriter();
+
+  /**
+   * Removes the elements in this LogStream prior to the given recordId.
+   *
+   * @param recordId A non-negative recordId for this LogStream
+   */
+  CompletableFuture truncateBefore(long recordId);
+
+  /**
+   * Returns the recordId which is the start of the LogStream. When there 
are records which were truncated
+   * from the LogStream, this will return a value larger than {@code 0}.
+   */
+  CompletableFuture getFirstRecordId();
+
+  /**
+   * Copies all records from the beginning of the LogStream until the 
given {@code recordId}
+   * to the configured archival storage.
+   *
+   * @param recordId A non-negative recordId for this LogStream
+   */
+  CompletableFuture archiveBefore(long recordId);
+
+  /**
+   * Returns the recordId, prior to which, all records in the LogStream 
are archived.
+   */
+  CompletableFuture getArchivalPoint();
--- End diff --

What does concept of log archive give us? We either need log (regardless 
its status: active/archive) or we do not.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-22 Thread VladRodionov
Github user VladRodionov commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212086396
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/AsyncLogReader.java
 ---
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Asynchronous client interface to read from a LogStream.
+ */
+public interface AsyncLogReader extends AutoCloseable {
+
+  /**
+   * Seeks to the position before the record at the provided {@code 
recordId} in the LogStream.
+   *
+   * @param recordId A non-negative, recordId in the LogStream
+   * @return A future for when the operation is completed.
+   */
+  CompletableFuture seek(long recordId) throws IOException;
+
+  /**
+   * Reads the next record from the LogStream at the current position and 
advances the current position
+   * to after the record which was just returned.
+   *
+   * @return A future providing the data for the next record.
+   */
+  CompletableFuture readNext() throws IOException;
+
+  /**
+   * Reads the next {@code numRecords} records from the LogStream, 
starting at the current position. This method
+   * may return fewer than requested records if the LogStream does not 
have sufficient records to return.
+   *
+   * @param numRecords The number of records to return
+   * @return A future providing the records, no more than the requested 
{@code numRecords} amount.
+   */
+  CompletableFuture> readBulk(int numRecords) throws 
IOException;
--- End diff --

Sure, let us do not breed buffers - they are expensive :)


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-22 Thread VladRodionov
Github user VladRodionov commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212086058
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/ArchivedLog.java 
---
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+/**
+ * A {@link LogStream} which has been archived in some external
+ * system. This interface is parameterized to allow for implementations
+ * to use their own class to encapsulate how to find the archived log.
+ *
+ * In the majority of cases, this should be transparent to end-users, as
+ * the {@link LogStream} should hide the fact that this even exists.
+ * TODO maybe that means this should be client-facing at all?
+ *
+ * @param  A referent to the log on the external system.
+ */
+public interface ArchivedLog extends AutoCloseable {
--- End diff --

We are not talking about full fledged FileSystem API here, but some API 
would help in adoption, taking into account familiar file system abstraction. 
Besides this, as a HBase developer I have my own requirements (see my comments 
above) to this Log Service and they can be fulfilled only by adopting FS-like 
API.   


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-22 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212083394
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/ArchivedLog.java 
---
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+/**
+ * A {@link LogStream} which has been archived in some external
+ * system. This interface is parameterized to allow for implementations
+ * to use their own class to encapsulate how to find the archived log.
+ *
+ * In the majority of cases, this should be transparent to end-users, as
+ * the {@link LogStream} should hide the fact that this even exists.
+ * TODO maybe that means this should be client-facing at all?
+ *
+ * @param  A referent to the log on the external system.
+ */
+public interface ArchivedLog extends AutoCloseable {
--- End diff --

Looking more closely at java.nio.file.Path, java.nio.file.WatchService, 
java.nio.file.FileSystem, and java.nio.file.spi.FileSystemProvider, I think 
that there are definitely pieces which make sense for us to use, but I think we 
also get a lot of baggage that we don't want (that we would just always have 
implemented with a RuntimeException).

The WatchService is basically what the RecordListener was meant to be. I 
think using a "filesystem path" (e.g. /a/b/c) does make sense and gives us 
flexibility for the LogService overall (instead of just "log name").

What about something in between? Take some of the good parts of Path and 
FileSystem, and pull them into LogService directly?


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-22 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212077861
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/AsyncLogReader.java
 ---
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Asynchronous client interface to read from a LogStream.
+ */
+public interface AsyncLogReader extends AutoCloseable {
+
+  /**
+   * Seeks to the position before the record at the provided {@code 
recordId} in the LogStream.
+   *
+   * @param recordId A non-negative, recordId in the LogStream
+   * @return A future for when the operation is completed.
+   */
+  CompletableFuture seek(long recordId) throws IOException;
+
+  /**
+   * Reads the next record from the LogStream at the current position and 
advances the current position
+   * to after the record which was just returned.
+   *
+   * @return A future providing the data for the next record.
+   */
+  CompletableFuture readNext() throws IOException;
+
+  /**
+   * Reads the next {@code numRecords} records from the LogStream, 
starting at the current position. This method
+   * may return fewer than requested records if the LogStream does not 
have sufficient records to return.
+   *
+   * @param numRecords The number of records to return
+   * @return A future providing the records, no more than the requested 
{@code numRecords} amount.
+   */
+  CompletableFuture> readBulk(int numRecords) throws 
IOException;
--- End diff --

@VladRodionov given your suggestion to have a `readNext(ByteBuffer)` 
variant, do you think that we should have a similar addition for `readBulk`, 
e.g. `readBulk(List)` where we fill each provided ByteBuffer with 
the next record? For example, if the caller gave a `List` of length 
5, we would return back 5 records.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-22 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212074817
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStream.java 
---
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A distributed log with "infinite" length that supports reads and writes.
+ */
+public interface LogStream {
+
+  /**
+   * Returns the unique name to identify this log.
+   */
+  LogName getName();
+
+  /**
+   * Returns the size of this LogStream in bytes.
+   */
+  long getSizeInBytes();
+
+  /**
+   * Returns the number of records in this LogStream.
+   */
+  long getSizeInRecords();
+
+  /**
+   * Creates a reader to read this LogStream asynchronously.
+   *
+   * @return An asynchronous reader
+   */
+  AsyncLogReader createAsyncReader();
+
+  /**
+   * Creates a writer to write to this LogStream asynchronously.
+   *
+   * @return An asynchronous writer
+   */
+  AsyncLogWriter createAsyncWriter();
+
+  /**
+   * Creates a reader to read this LogStream synchronously.
+   *
+   * @return A synchronous reader
+   */
+  LogReader createReader();
+
+  /**
+   * Creates a write to write to this LogStream synchronously.
+   *
+   * @return A synchronous writer
+   */
+  LogWriter createWriter();
+
+  /**
+   * Removes the elements in this LogStream prior to the given recordId.
+   *
+   * @param recordId A non-negative recordId for this LogStream
+   */
+  CompletableFuture truncateBefore(long recordId);
+
+  /**
+   * Returns the recordId which is the start of the LogStream. When there 
are records which were truncated
+   * from the LogStream, this will return a value larger than {@code 0}.
+   */
+  CompletableFuture getFirstRecordId();
+
+  /**
+   * Copies all records from the beginning of the LogStream until the 
given {@code recordId}
+   * to the configured archival storage.
+   *
+   * @param recordId A non-negative recordId for this LogStream
+   */
+  CompletableFuture archiveBefore(long recordId);
--- End diff --

On second thought "archiveBefore" would still be correct. I still like the 
inclusivity.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-22 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212009732
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStream.java 
---
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A distributed log with "infinite" length that supports reads and writes.
+ */
+public interface LogStream {
+
+  /**
+   * Returns the unique name to identify this log.
+   */
+  LogName getName();
+
+  /**
+   * Returns the size of this LogStream in bytes.
+   */
+  long getSizeInBytes();
+
+  /**
+   * Returns the number of records in this LogStream.
+   */
+  long getSizeInRecords();
+
+  /**
+   * Creates a reader to read this LogStream asynchronously.
+   *
+   * @return An asynchronous reader
+   */
+  AsyncLogReader createAsyncReader();
+
+  /**
+   * Creates a writer to write to this LogStream asynchronously.
+   *
+   * @return An asynchronous writer
+   */
+  AsyncLogWriter createAsyncWriter();
+
+  /**
+   * Creates a reader to read this LogStream synchronously.
+   *
+   * @return A synchronous reader
+   */
+  LogReader createReader();
+
+  /**
+   * Creates a write to write to this LogStream synchronously.
+   *
+   * @return A synchronous writer
+   */
+  LogWriter createWriter();
+
--- End diff --

My hope was that we could support multiple, independent writers. The writer 
implementation would be able to hide most details (I hope).


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-22 Thread joshelser
Github user joshelser commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r212009110
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/AsyncLogReader.java
 ---
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Asynchronous client interface to read from a LogStream.
+ */
+public interface AsyncLogReader extends AutoCloseable {
+
+  /**
+   * Seeks to the position before the record at the provided {@code 
recordId} in the LogStream.
+   *
+   * @param recordId A non-negative, recordId in the LogStream
+   * @return A future for when the operation is completed.
+   */
+  CompletableFuture seek(long recordId) throws IOException;
+
--- End diff --

If you think that would be helpful, we can. I am wondering about the 
frequency of use of such an API. Does it make sense to have that "wrapper" when 
we have something like `seek(getFirstRecordId())` already present? I don't feel 
strongly either way :)


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-21 Thread VladRodionov
Github user VladRodionov commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211684885
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogReader.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Synchronous client interface to read from a LogStream.
+ */
+public interface LogReader extends AutoCloseable {
+
+  /**
+   * Seeks to the position before the record at the provided {@code 
offset} in the LogStream.
+   *
+   * @param offset A non-negative, offset in the LogStream
+   * @return A future for when the operation is completed.
+   */
+  void seek(long recordId) throws IOException;
+
+  /**
+   * Reads the next record from the LogStream at the current position and 
advances the current position
+   * to after the record which was just returned.
+   *
+   * @return The data for the next record.
+   */
+  ByteBuffer readNext() throws IOException;
--- End diff --

readNext(ByteBuffer) -  to reuse existing buffer


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-21 Thread VladRodionov
Github user VladRodionov commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211684524
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogName.java ---
@@ -0,0 +1,28 @@
+package org.apache.ratis.logservice.api;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Identifier to uniquely identify a {@link LogStream}.
+ */
+public class LogName {
--- End diff --

LogName -> java.io.Path. Let us adopt file system for log service


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-17 Thread ankitsinghal
Github user ankitsinghal commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211064459
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStream.java 
---
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A distributed log with "infinite" length that supports reads and writes.
+ */
+public interface LogStream {
+
+  /**
+   * Returns the unique name to identify this log.
+   */
+  LogName getName();
+
+  /**
+   * Returns the size of this LogStream in bytes.
+   */
+  long getSizeInBytes();
+
+  /**
+   * Returns the number of records in this LogStream.
+   */
+  long getSizeInRecords();
+
+  /**
+   * Creates a reader to read this LogStream asynchronously.
+   *
+   * @return An asynchronous reader
+   */
+  AsyncLogReader createAsyncReader();
+
+  /**
+   * Creates a writer to write to this LogStream asynchronously.
+   *
+   * @return An asynchronous writer
+   */
+  AsyncLogWriter createAsyncWriter();
+
+  /**
+   * Creates a reader to read this LogStream synchronously.
+   *
+   * @return A synchronous reader
+   */
+  LogReader createReader();
+
+  /**
+   * Creates a write to write to this LogStream synchronously.
+   *
+   * @return A synchronous writer
+   */
+  LogWriter createWriter();
+
+  /**
+   * Removes the elements in this LogStream prior to the given recordId.
+   *
+   * @param recordId A non-negative recordId for this LogStream
+   */
+  CompletableFuture truncateBefore(long recordId);
+
+  /**
+   * Returns the recordId which is the start of the LogStream. When there 
are records which were truncated
+   * from the LogStream, this will return a value larger than {@code 0}.
+   */
+  CompletableFuture getFirstRecordId();
+
--- End diff --

Similarly getlastRecordId() would be helpfull.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-17 Thread ankitsinghal
Github user ankitsinghal commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211064158
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogService.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Entry point for interacting with the Ratis LogService.
+ */
+public interface LogService {
+
+  /**
+   * Creates a new {@link LogStream} identified by the given name. Throws
+   * an exception if a {@link LogStream} with the given name already 
exists.
+   *
+   * @param name Unique name for this LogStream.
+   */
+  CompletableFuture createLog(LogName name);
+
+  /**
+   * Fetches the {@link LogStream} identified by the given name.
+   *
+   * @param name The name of the LogStream
+   */
+  CompletableFuture getLog(LogName name);
+
+  /**
+   * Lists all {@link LogStream} instances known by this LogService.
+   */
+  CompletableFuture> listLogs();
--- End diff --

can accept a regular expression to avoid large data requested from the 
service.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-17 Thread ankitsinghal
Github user ankitsinghal commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211064553
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/AsyncLogReader.java
 ---
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Asynchronous client interface to read from a LogStream.
+ */
+public interface AsyncLogReader extends AutoCloseable {
+
+  /**
+   * Seeks to the position before the record at the provided {@code 
recordId} in the LogStream.
+   *
+   * @param recordId A non-negative, recordId in the LogStream
+   * @return A future for when the operation is completed.
+   */
+  CompletableFuture seek(long recordId) throws IOException;
+
--- End diff --

Can add reset() also to reset according to the logStream getFirstRecordId().


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-17 Thread ankitsinghal
Github user ankitsinghal commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211064260
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogWriter.java 
---
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Synchronous client interface to write to a LogStream.
+ */
+public interface LogWriter extends AutoCloseable {
+
+  /**
+   * Appends the given data as a record in the LogStream.
+   *
+   * @param data The record to append
+   * @return The recordId for the record just written
+   */
+  long write(ByteBuffer data) throws IOException;
+
+  /**
+   * Guarantees that all previous data appended by {@link 
#write(ByteBuffer)} are persisted
+   * and durable in the LogStream.
+   *
+   * @return TODO Unknown?
+   */
--- End diff --

Return last recordId committed.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-17 Thread ankitsinghal
Github user ankitsinghal commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211064684
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogService.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Entry point for interacting with the Ratis LogService.
+ */
+public interface LogService {
+
+  /**
+   * Creates a new {@link LogStream} identified by the given name. Throws
+   * an exception if a {@link LogStream} with the given name already 
exists.
+   *
+   * @param name Unique name for this LogStream.
+   */
+  CompletableFuture createLog(LogName name);
--- End diff --

While creating log , we may need to add some metadata for the stream, for 
eg:- Details of SERDE classes, compression etc.
And need the corresponding API in LogStream to get the MetaData.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-17 Thread ankitsinghal
Github user ankitsinghal commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r21106
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogStream.java 
---
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A distributed log with "infinite" length that supports reads and writes.
+ */
+public interface LogStream {
+
+  /**
+   * Returns the unique name to identify this log.
+   */
+  LogName getName();
+
+  /**
+   * Returns the size of this LogStream in bytes.
+   */
+  long getSizeInBytes();
+
+  /**
+   * Returns the number of records in this LogStream.
+   */
+  long getSizeInRecords();
+
+  /**
+   * Creates a reader to read this LogStream asynchronously.
+   *
+   * @return An asynchronous reader
+   */
+  AsyncLogReader createAsyncReader();
+
+  /**
+   * Creates a writer to write to this LogStream asynchronously.
+   *
+   * @return An asynchronous writer
+   */
+  AsyncLogWriter createAsyncWriter();
+
+  /**
+   * Creates a reader to read this LogStream synchronously.
+   *
+   * @return A synchronous reader
+   */
+  LogReader createReader();
+
+  /**
+   * Creates a write to write to this LogStream synchronously.
+   *
+   * @return A synchronous writer
+   */
+  LogWriter createWriter();
+
--- End diff --

I believe the plan is to support a single writer(otherwise commit() would 
get tedious to manage at server), right ? can we make it getWriter()?


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-17 Thread ankitsinghal
Github user ankitsinghal commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211064777
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/ArchivedLog.java 
---
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+/**
+ * A {@link LogStream} which has been archived in some external
+ * system. This interface is parameterized to allow for implementations
+ * to use their own class to encapsulate how to find the archived log.
+ *
+ * In the majority of cases, this should be transparent to end-users, as
+ * the {@link LogStream} should hide the fact that this even exists.
+ * TODO maybe that means this should be client-facing at all?
+ *
--- End diff --

Yes, it should not be client-facing at all. Just knowing from logStream 
that if I'm reading from archived or active location should be 
fine(recordId

[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-17 Thread ankitsinghal
Github user ankitsinghal commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r211064058
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogReader.java 
---
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Synchronous client interface to read from a LogStream.
+ */
+public interface LogReader extends AutoCloseable {
+
--- End diff --

Need a way to get the recordId of the current record read 
(getCurrentRecordId()?), it will be needed to keep a track so that we can start 
from where we left after any failure (using seek)


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-16 Thread chrajeshbabu
Github user chrajeshbabu commented on a diff in the pull request:

https://github.com/apache/incubator-ratis/pull/4#discussion_r210808962
  
--- Diff: 
ratis-logservice/src/main/java/org/apache/ratis/logservice/api/LogWriter.java 
---
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.logservice.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Synchronous client interface to write to a LogStream.
+ */
+public interface LogWriter extends AutoCloseable {
+
+  /**
+   * Appends the given data as a record in the LogStream.
+   *
+   * @param data The record to append
+   * @return The recordId for the record just written
+   */
+  long write(ByteBuffer data) throws IOException;
+
+  /**
+   * Guarantees that all previous data appended by {@link 
#write(ByteBuffer)} are persisted
+   * and durable in the LogStream.
+   *
+   * @return TODO Unknown?
+   */
+  long commit() throws IOException;
--- End diff --

@joshelser This need to be changed to sync.


---


[GitHub] incubator-ratis pull request #4: RATIS-272 logservice api

2018-08-16 Thread joshelser
GitHub user joshelser opened a pull request:

https://github.com/apache/incubator-ratis/pull/4

RATIS-272 logservice api

WIP. Based on Rajeshbabu's first API suggestions. Also adds:

* LogStream and LogService interfaces
* LogName to uniquely identify a LogStream in a LogService
* Listener (Tail) API for a LogStream

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/joshelser/incubator-ratis 272-logservice-api

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/incubator-ratis/pull/4.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4


commit 2a0465c06bdbe718824d6130426b352445b36f0a
Author: Josh Elser 
Date:   2018-08-16T16:34:23Z

RATIS-272 First wag at an API for the LogService




---