[ 
https://issues.apache.org/jira/browse/CASSANDRA-14062?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16279027#comment-16279027
 ] 

Rei Odaira commented on CASSANDRA-14062:
----------------------------------------

In the current implementation, our replayer accesses the following Cassandra 
classes:

import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.Mutation;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.commitlog.CommitLogPosition;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.db.rows.SerializationHelper;
import org.apache.cassandra.io.util.DataInputBuffer;
import org.apache.cassandra.io.util.RebufferingInputStream;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.WrappedRunnable;

I have to admit that so far we have never tried to minimize these dependencies. 
 Probably we can share some logic with 
org.apache.cassandra.db.commitlog.CommitLogReplayer, so that we depend mostly 
on CommitLogReplayer and not much on other classes.
https://github.com/ppc64le/capi-rowcache/blob/capi-commitlog/src/com/ibm/capiflash/cassandra/commitlog/FlashBulkReplayer.java

> Pluggable CommitLog
> -------------------
>
>                 Key: CASSANDRA-14062
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14062
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: Rei Odaira
>            Assignee: Rei Odaira
>              Labels: features
>             Fix For: 4.x
>
>         Attachments: pluggable-commitlog-src.patch, 
> pluggable-commitlog-test.patch
>
>
> This proposal is to make CommitLog pluggable, as discussed in [the Cassandra 
> dev mailing 
> list|https://lists.apache.org/thread.html/1936194d86f5954fa099ced9a0733458eb3249bff3fae3e03e2d1bd8@%3Cdev.cassandra.apache.org%3E].
> We are developing a Cassandra plugin to store CommitLog on our low-latency 
> Flash device (CAPI-Flash). To do that, the original CommitLog interface must 
> be changed to allow plugins. Synching to CommitLog is one of the performance 
> bottlenecks in Cassandra especially with batch commit. I think the pluggable 
> CommitLog will allow other interesting alternatives, such as one using SPDK.
> Our high-level design is similar to the CacheProvider framework
> in org.apache.cassandra.cache:
> * Introduce a new interface, ICommitLog, with methods like 
> getCurrentPosition(), add(), shutdownBlocking(), etc.
> * CommitLog implements ICommitLog.
> * Introduce a new interface, CommitLogProvider, with a create() method, 
> returning ICommitLog.
> * Introduce a new class FileCommitLogProvider implementing CommitLogProvider, 
> to return a singleton instance of CommitLog.
> * Introduce a new property in cassandra.yaml, commitlog_class_name, which 
> specifies what CommitLogProvider to use.  The default is 
> FileCommitLogProvider.
> * Introduce a new class, CommitLogHelper, that loads the class specified by 
> the commitlog_class_name property, creates an instance, and stores it to 
> CommitLogHelper.instance.
> * Replace all of the references to CommitLog.instance with 
> CommitLogHelper.instance.
> Attached are two patches. "pluggable-commitlog-src.patch" is for changes in 
> the src directory, and "pluggable-commitlog-test.patch" is for the test 
> directory.



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

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to