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

David Capwell commented on CASSANDRA-17103:
-------------------------------------------

Review:

* General: you have a few "TODO (now)" remaining, should those be cleaned up?
* accord-core/src/main/java/accord/api/Read.java
ReadFuture.accept should "return" after calling "tryFailure"
* accord-core/src/main/java/accord/local/CommandStores.java
"forEach" takes both a "TxnOperation" and "Keys", but "TxnOperation" holds ref 
to "Keys", so shouldn't we drop "Keys"?  Looking at the usage, it seems like we 
could...

{code}
node.forEachLocalSince(TxnOperation.scopeFor(txnId), full.txn.keys(), ...
{code}

{code}
TxnOperation scope = TxnOperation.scopeFor(txnId, max.txn.keys());
...
node.forEachLocalSince(scope, max.txn.keys(), ...
{code}
This change trickled throughout a lot of different places, so would be good to 
grasp the expected semantics... if the TxnOperation has keys, and you provide 
keys... "should" they be released or "must" they be related? or something else?
* accord-core/src/main/java/accord/coordinate/Execute.java
We are not good with nullability in Accord, so its not clear if read is allowed 
to be null... Based off Caleb's branch 
org.apache.cassandra.cql3.statements.TransactionStatement#createRead won't be 
null in a write only txn, but is null allowed or not?  If so then we need a 
null check at "if (txn.read().keys().isEmpty())"
* accord-core/src/main/java/accord/impl/InMemoryCommand.java
eq is missing homeKey, progressKey, isGloballyPersistent, is this intentional?
* accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
I still need to look closer at our epoch logic but if we have multiple epochs 
that are not full acknowledged then wouldn't forEpochCommands have the wrong 
min epoch?  This is only used in tests and not in the C* patch... if this is 
only for tests then I am cool with this, but it would be good to start 
annotating what methods are not meant to be used in `src/main`
Same comment for forCommittedInEpoch
Nit: forCommittedInEpoch and forEpochCommands are mostly copy/paste of each 
other, can we create a new util function such as

{code}
public void forEach(KeyRanges ranges, long epoch, Consumer<CommandsForKey> 
consumer)
    {
        Timestamp minTimestamp = new Timestamp(epoch, Long.MIN_VALUE, 
Integer.MIN_VALUE, Node.Id.NONE);
        Timestamp maxTimestamp = new Timestamp(epoch, Long.MAX_VALUE, 
Integer.MAX_VALUE, Node.Id.MAX);
        for (KeyRange range : ranges)
        {
            Iterable<CommandsForKey> rangeCommands = 
commandsForKey.subMap(range.start(),
                                                                           
range.startInclusive(),
                                                                           
range.end(),
                                                                           
range.endInclusive()).values();
            rangeCommands.forEach(consumer);
        }
    }
{code}

nit: "protected void processInternal(Consumer<? super CommandStore> consumer, 
Promise<Void> promise)" could be the following to avoid copy/paste

{code}
protected void processInternal(Consumer<? super CommandStore> consumer, 
Promise<Void> promise)
    {
        processInternal(cs -> {
            consumer.accept(cs);
            return null;
        }, promise);
    }
{code}

Synchronized doesn't use "synchronized" in every method... so name feels off...
SingleThread should expose ability to control thread name... in C* we have a 
"global prefix" for jvm-dtest 
nit: assertThread, can we add a useful message?  When something like this fails 
its too hard to figure out what's going on, so we should be nice and provide a 
useful error.

* accord-core/src/main/java/accord/impl/InMemoryCommandStores.java
It would be nice to move "forEachLocal" to lower level... seems that we try to 
overload fold in cases where forEach would be simpler.

* accord-core/src/main/java/accord/impl/InMemoryCommandsForKey.java
InMemoryCommandTimeseries.add should check to see if a conflict was detected

* accord-core/src/main/java/accord/impl/SimpleProgressLog.java
For consistency shouldn't notPersisted be private with a getter?  this patch 
switches many things to getters yet makes this public field access.

* accord-core/src/main/java/accord/local/Command.java
nit: given the refactor to make all state abstract wouldn't it be best if this 
became an Interface?
apply now returns a Future but some code paths ignore the output, so if apply 
fails or is async, there may be unexpected results?  Examples: 
accord.coordinate.CheckOnCommitted#onSuccessCriteriaOrExhaustion(accord.messages.CheckStatus.CheckStatusOkFull),
 and accord.impl.SimpleProgressLog.ApplyAndCheck#process

If apply is still being "applied" then the call to "commit" may cause a race 
condition as it could be called while the writes are happening, leading to the 
following timeline:

{code}
T1 apply -> sets status to "Executed", send writes to be written in the 
background
T2 commit -> set status to "Committed"
T3 apply complete, call "postApply" -> set status "Applied"
{code}

specifically this comment applies to 
"accord.coordinate.CheckOnCommitted#onSuccessCriteriaOrExhaustion(accord.messages.CheckStatus.CheckStatusOkFull)"...
 I might not be grasping the "epoch - 1" part

* accord-core/src/main/java/accord/messages/Apply.java
nit: "waitAndReduce" throws on future error rather than return a failed future, 
this feels off to me but looks to be because "mapReduceLocal" doesn't allow 
changing the return type; we want Void in this context but that API doesn't 
allow... would it be better to allow a different return type?  Caller never 
checks return so Future<?> is brittle...
* new classes need a license
{code}
/Users/dcapwell/src/github/apache/cassandra-accord/team/bdeggleston/accord-core/src/main/java/accord/impl/InMemoryCommandsForKey.java
/Users/dcapwell/src/github/apache/cassandra-accord/team/bdeggleston/accord-core/src/main/java/accord/impl/InMemoryCommandStore.java
/Users/dcapwell/src/github/apache/cassandra-accord/team/bdeggleston/accord-core/src/main/java/accord/impl/InMemoryCommandStores.java
/Users/dcapwell/src/github/apache/cassandra-accord/team/bdeggleston/accord-core/src/main/java/accord/impl/InMemoryCommand.java
/Users/dcapwell/src/github/apache/cassandra-accord/team/bdeggleston/accord-core/src/main/java/accord/utils/Utils.java
/Users/dcapwell/src/github/apache/cassandra-accord/team/bdeggleston/accord-core/src/main/java/accord/utils/VisibleForImplementation.java
/Users/dcapwell/src/github/apache/cassandra-accord/team/bdeggleston/accord-core/src/main/java/accord/local/TxnOperation.java
{code}

* 
accord-core/src/main/java/org/apache/cassandra/utils/concurrent/FutureCombiner.java
please remove in favor of 
https://github.com/bdeggleston/cassandra-accord/pull/2, this breaks C* 
integration

> CEP-15 (C*): Messaging and storage engine integration
> -----------------------------------------------------
>
>                 Key: CASSANDRA-17103
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-17103
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Accord
>            Reporter: Benedict Elliott Smith
>            Priority: Normal
>
> This work encompasses implementing Accord’s storage and networking interfaces 
> within Cassandra, so that messages may be sent around the cluster and 
> exectuted



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to