dcapwell commented on code in PR #50:
URL: https://github.com/apache/cassandra-accord/pull/50#discussion_r1260343014


##########
accord-core/src/main/java/accord/impl/SimpleProgressLog.java:
##########
@@ -483,59 +330,71 @@ void record(Known known)
                 }
 
                 @Override
-                void run(Command command)
+                void run(SafeCommandStore safeStore, SafeCommand safeCommand)
                 {
-                    if (command.has(blockedUntil))
+                    Command command = safeCommand.current();
+                    if (command.saveStatus().execution.compareTo(blockedUntil) 
>= 0)

Review Comment:
   nit:
   
   ```suggestion
                       if (command.isAtLeast(blockedUntil))
   ```



##########
accord-core/src/main/java/accord/local/SaveStatus.java:
##########
@@ -35,38 +44,122 @@
  */
 public enum SaveStatus
 {
-    NotWitnessed                    (Status.NotWitnessed),
+    // TODO (expected): erase Uninitialised in Context once command finishes
+    // TODO (expected): we can use Uninitialised in several places to 
simplify/better guarantee correct behaviour with truncation
+    Uninitialised                   (Status.NotDefined),
+    NotDefined                      (Status.NotDefined),
     PreAccepted                     (Status.PreAccepted),
     AcceptedInvalidate              (Status.AcceptedInvalidate),
-    AcceptedInvalidateWithDefinition(Status.AcceptedInvalidate,    
DefinitionKnown,   ExecuteAtUnknown,  DepsUnknown,  OutcomeUnknown),
+    AcceptedInvalidateWithDefinition(Status.AcceptedInvalidate,    
DefinitionKnown,   ExecuteAtUnknown,  DepsUnknown,  Unknown),
     Accepted                        (Status.Accepted),
-    AcceptedWithDefinition          (Status.Accepted,              
DefinitionKnown,   ExecuteAtProposed, DepsProposed, OutcomeUnknown),
+    AcceptedWithDefinition          (Status.Accepted,              
DefinitionKnown,   ExecuteAtProposed, DepsProposed, Unknown),
     PreCommitted                    (Status.PreCommitted),
-    PreCommittedWithAcceptedDeps    (Status.PreCommitted,          
DefinitionUnknown, ExecuteAtKnown,    DepsProposed, OutcomeUnknown),
-    PreCommittedWithDefinition      (Status.PreCommitted,          
DefinitionKnown,   ExecuteAtKnown,    DepsUnknown,  OutcomeUnknown),
-    PreCommittedWithDefinitionAndAcceptedDeps(Status.PreCommitted, 
DefinitionKnown,   ExecuteAtKnown,    DepsProposed, OutcomeUnknown),
+    PreCommittedWithAcceptedDeps    (Status.PreCommitted,          
DefinitionUnknown, ExecuteAtKnown,    DepsProposed, Unknown),
+    PreCommittedWithDefinition      (Status.PreCommitted,          
DefinitionKnown,   ExecuteAtKnown,    DepsUnknown,  Unknown),
+    PreCommittedWithDefinitionAndAcceptedDeps(Status.PreCommitted, 
DefinitionKnown,   ExecuteAtKnown,    DepsProposed, Unknown),
     Committed                       (Status.Committed),
-    ReadyToExecute                  (Status.ReadyToExecute),
+    ReadyToExecute                  (Status.ReadyToExecute,                    
                                                                
LocalExecution.ReadyToExecute),
     PreApplied                      (Status.PreApplied),
-    Applied                         (Status.Applied),
-    Invalidated                     (Status.Invalidated);
-    
+    Applying                        (Status.PreApplied),
+    Applied                         (Status.Applied,                           
                                                                
LocalExecution.Applied),
+    TruncatedApplyWithDeps          (Status.Truncated,             
DefinitionUnknown, ExecuteAtKnown,    DepsKnown,    Outcome.Apply,          
LocalExecution.CleaningUp),
+    TruncatedApplyWithOutcome       (Status.Truncated,             
DefinitionUnknown, ExecuteAtKnown,    DepsUnknown,  Outcome.Apply,          
LocalExecution.CleaningUp),
+    TruncatedApply                  (Status.Truncated,             
DefinitionUnknown, ExecuteAtKnown,    DepsUnknown,  Outcome.WasApply,       
LocalExecution.CleaningUp),
+    Erased                          (Status.Truncated,             
DefinitionUnknown, ExecuteAtUnknown,  DepsUnknown,  Outcome.Erased,         
LocalExecution.CleaningUp),
+    Invalidated                     (Status.Invalidated,                       
                                                                
LocalExecution.CleaningUp),
+    ;
+
+    public enum LocalExecution
+    {
+        NotReady(Nothing),
+        ReadyToExclude(ExecuteAtOnly),
+        WaitingToExecute(Decision),
+        ReadyToExecute(Decision),
+        WaitingToApply(Apply),
+        Applying(Apply),
+        Applied(Apply),
+        CleaningUp(Nothing);
+
+        public final Known requires;
+        LocalExecution(Known requires)

Review Comment:
   can you add a space from the above?



##########
accord-core/src/test/java/accord/utils/SimpleBitSetTest.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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 accord.utils;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Random;
+import java.util.function.BiConsumer;
+import java.util.function.IntConsumer;
+
+import com.google.common.collect.Lists;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class SimpleBitSetTest
+{
+    private static final int NOT_FOUND = Integer.MAX_VALUE;
+
+    private static class Check
+    {
+        final SimpleBitSet test;
+        final BitSet canon;
+        final int size;
+
+        private Check(SimpleBitSet test, BitSet canon, int size)
+        {
+            this.test = test;
+            this.canon = canon;
+            this.size = size;
+        }
+
+        void check(Random random)
+        {
+            assertEquals(canon.cardinality(), test.setBitCount());
+            assertEquals(canon.nextSetBit(0), test.firstSetBit());
+            assertEquals(normaliseNotFound(canon.nextSetBit(0)), 
test.firstSetBit(NOT_FOUND));
+            assertEquals(canon.previousSetBit(size), test.lastSetBit());
+            assertEquals(normaliseNotFound(canon.previousSetBit(size)), 
test.lastSetBit(NOT_FOUND));
+
+            forIndices(random, i -> assertEquals(canon.nextSetBit(i), 
test.nextSetBit(i)));
+            forIndices(random, i -> 
assertEquals(normaliseBefore(canon.nextSetBit(0), i), 
test.firstSetBitBefore(i)));
+            forIndices(random, i -> 
assertEquals(normaliseNotFoundBefore(canon.nextSetBit(0), i), 
test.firstSetBitBefore(i, NOT_FOUND)));
+            forRanges(random, (i, j) -> 
assertEquals(normaliseBefore(canon.nextSetBit(i), j), test.nextSetBitBefore(i, 
j)));
+            forRanges(random, (i, j) -> 
assertEquals(normaliseNotFoundBefore(canon.nextSetBit(i), j), 
test.nextSetBitBefore(i, j, NOT_FOUND)));
+
+            forIndices(random, i -> assertEquals(i == 0 ? -1 : 
canon.previousSetBit(i - 1), test.prevSetBit(i)));
+            forIndices(random, i -> assertEquals(normaliseNotBefore(size == 0 
? -1 : canon.previousSetBit(size - 1), i), test.lastSetBitNotBefore(i)));
+            forIndices(random, i -> 
assertEquals(normaliseNotFoundNotBefore(size == 0 ? -1 : 
canon.previousSetBit(size - 1), i), test.lastSetBitNotBefore(i, NOT_FOUND)));
+            forRanges(random, (i, j) -> assertEquals(normaliseNotBefore(j == 0 
? -1 : canon.previousSetBit(j - 1), i), test.prevSetBitNotBefore(j, i)));
+            forRanges(random, (i, j) -> 
assertEquals(normaliseNotFoundNotBefore(j == 0 ? -1 : canon.previousSetBit(j - 
1), i), test.prevSetBitNotBefore(j, i, NOT_FOUND)));
+
+            List<Integer> canonCollect = new ArrayList<>(), testCollect = new 
ArrayList<>();
+            canon.stream().forEachOrdered(canonCollect::add);
+            test.forEach(testCollect, List::add);
+            assertEquals(canonCollect, testCollect);
+
+            canonCollect = Lists.reverse(canonCollect);
+            testCollect.clear();
+            test.reverseForEach(testCollect, List::add);
+            assertEquals(canonCollect, testCollect);
+        }
+
+        void forIndices(Random random, IntConsumer consumer)
+        {
+            for (int c = 0 ; c < 100 ; ++c)
+            {
+                int i = random.nextInt(size + 1);
+                consumer.accept(i);
+            }
+        }
+
+        void forRanges(Random random, BiConsumer<Integer, Integer> consumer)
+        {
+            for (int c = 0 ; c < 100 ; ++c)
+            {
+                int i = random.nextInt(size + 1);
+                int j = random.nextInt(size + 1);
+                if (i > j) { int t = i; i = j; j = t; }
+                consumer.accept(i, j);
+            }
+        }
+
+        static Check generate(Random random, int maxSize, int modCount, int 
runLength, float runChance, float clearChance)
+        {
+            int size = random.nextInt(maxSize);
+            runLength = Math.min(size, runLength);
+            BitSet canon = new BitSet(size);
+            SimpleBitSet test = new SimpleBitSet(size);
+            if (size > 0)
+            {
+                while (modCount-- > 0)
+                {
+                    boolean set = random.nextFloat() >= clearChance;
+                    boolean run = runLength > 1 && random.nextFloat() < 
runChance;
+                    if (run)
+                    {
+                        int i = random.nextInt(size);
+                        int j = random.nextInt(size);
+                        if (j < i) { int t = i; i = j; j = t; }
+                        j = Math.min(i + 1 + random.nextInt(runLength - 1), j);
+
+                        if (set)
+                        {
+                            canon.set(i, j);
+                            test.setRange(i, j);
+                        }
+                        else
+                        {
+                            canon.clear(i, j);
+                            while (i < j)
+                                test.unset(i++);
+                        }
+                    }
+                    else
+                    {
+                        int i = random.nextInt(size);
+                        if (set)
+                        {
+                            assertEquals(!canon.get(i), test.set(i));
+                            canon.set(i);
+                        }
+                        else
+                        {
+                            assertEquals(canon.get(i), test.unset(i));
+                            canon.clear(i);
+                        }
+                    }
+                    assertEquals(canon.cardinality(), test.setBitCount());
+                }
+            }
+            return new Check(test, canon, size);
+        }
+    }
+
+    @Test
+    public void testRandomBitSets()
+    {
+        Random random = new Random();
+        long seed = random.nextLong();
+        System.err.println("Seed: " + seed);
+        random.setSeed(seed);
+        testRandomBitSets(random, 100000);

Review Comment:
   nit: you can replace this with 
`qt().withExamples(100000).forAll(Gens.random()).check(SimpleBitSetTest::testRandomBitSet);`.
  The advantage is that failures will make sure the seed is captured in the 
JUnit report (this will not).  This is also a `pure` test, so if a failure *is* 
detected, the seed for the failing case is reported, so you can add a break 
point and not worry about skipping 10k cases



##########
accord-core/src/main/java/accord/utils/DeterministicSet.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 accord.utils;
+
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+import com.google.common.collect.Iterables;
+
+public class DeterministicSet<T> extends AbstractSet<T>
+{
+    static class Entry<T>
+    {
+        final T item;
+        Entry<T> prev;
+        Entry<T> next;
+
+        Entry(T item)
+        {
+            this.item = item;
+        }
+    }
+
+    // TODO (low priority): an identity hash map that doesn't mind concurrent 
modification / iteration
+    final Map<T, Entry<T>> lookup;
+    final Entry<T> head = new Entry<T>(null);
+
+    public DeterministicSet()
+    {
+        this(0);
+    }
+
+    public DeterministicSet(int size)
+    {
+        head.prev = head.next = head;
+        lookup = new HashMap<>((size * 4)/3);
+    }
+
+    public DeterministicSet(DeterministicSet<T> copy)
+    {
+        this(copy.size());
+        copy.forEach(this::addInternal);
+    }
+
+    DeterministicSet(Map<T, Entry<T>> lookup)
+    {
+        Invariants.checkArgument(lookup.isEmpty());
+        head.prev = head.next = head;
+        this.lookup = lookup;
+    }
+
+    @Override
+    public Iterator<T> iterator()
+    {
+        return new Iterator<T>()
+        {
+            boolean hasComputedNext = true;
+            Entry<T> next = head.next;

Review Comment:
   we detect the change during the iteration, but if we remove the `.next` 
before the iteration (`hasNext`) then this has a different behavior as it 
returns something removed



##########
accord-core/src/main/java/accord/utils/DeterministicSet.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 accord.utils;
+
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+import com.google.common.collect.Iterables;
+
+public class DeterministicSet<T> extends AbstractSet<T>
+{
+    static class Entry<T>
+    {
+        final T item;
+        Entry<T> prev;
+        Entry<T> next;
+
+        Entry(T item)
+        {
+            this.item = item;
+        }
+    }
+
+    // TODO (low priority): an identity hash map that doesn't mind concurrent 
modification / iteration
+    final Map<T, Entry<T>> lookup;
+    final Entry<T> head = new Entry<T>(null);
+
+    public DeterministicSet()
+    {
+        this(0);
+    }
+
+    public DeterministicSet(int size)
+    {
+        head.prev = head.next = head;
+        lookup = new HashMap<>((size * 4)/3);
+    }
+
+    public DeterministicSet(DeterministicSet<T> copy)
+    {
+        this(copy.size());
+        copy.forEach(this::addInternal);
+    }
+
+    DeterministicSet(Map<T, Entry<T>> lookup)
+    {
+        Invariants.checkArgument(lookup.isEmpty());
+        head.prev = head.next = head;
+        this.lookup = lookup;
+    }
+
+    @Override
+    public Iterator<T> iterator()
+    {
+        return new Iterator<T>()
+        {
+            boolean hasComputedNext = true;
+            Entry<T> next = head.next;
+            @Override
+            public boolean hasNext()
+            {
+                if (!hasComputedNext)
+                {
+                    // apply any deletion before deciding if hasNext
+                    while (next.next == null)
+                        next = next.prev;
+                    next = next.next;
+                    hasComputedNext = true;
+                }
+                return next != head;
+            }
+
+            @Override
+            public T next()
+            {
+                if (!hasNext())
+                    throw new NoSuchElementException();
+                T result = next.item;
+                // defer filtering of deleted items until hasNext(), so 
processing of next() can have applied
+                hasComputedNext = false;
+                return result;
+            }
+        };
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (!(o instanceof Iterable))
+            return false;
+
+        return Iterables.elementsEqual(this, (Iterable<?>) o);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = 1;
+
+        for (T element : this)
+            result = 31 * result + (element == null ? 0 : element.hashCode());
+
+        return result;
+    }
+
+    @Override
+    public int size()
+    {
+        return lookup.size();
+    }
+
+    boolean addInternal(T item)
+    {
+        Entry<T> entry = lookup.computeIfAbsent(item, Entry::new);
+        if (entry.prev != null)
+            return false;
+        entry.prev = head;
+        entry.next = head.next;
+        head.next = entry;
+        entry.next.prev = entry;
+        return true;
+    }
+    // we add to the front, and iterate in reverse order, so that we can add 
and remove while iterating without modifying the set we iterate over

Review Comment:
   add a space between this and line 144



##########
accord-core/src/main/java/accord/messages/ReadTxnData.java:
##########
@@ -218,6 +232,14 @@ void maybeRead(SafeCommandStore safeStore, SafeCommand 
safeCommand)
         }
     }
 
+    @Override
+    protected synchronized void readComplete(CommandStore commandStore, 
@Nullable Data result, @Nullable Ranges unavailable)
+    {
+        // TODO (expected): lots of undesirable costs associated with the 
obsoletion tracker
+//        commandStore.execute(contextFor(txnId), safeStore -> 
safeStore.command(txnId).removeListener(obsoleteTracker));

Review Comment:
   > Why?
   
   because we normally remove commented out code...  I am +0 if you want to 
keep it, but feel its still best to remove



##########
accord-core/src/main/java/accord/messages/BeginInvalidation.java:
##########
@@ -61,13 +58,14 @@ public void process()
     }
 
     @Override
-    public InvalidateReply apply(SafeCommandStore instance)
+    public InvalidateReply apply(SafeCommandStore safeStore)
     {
-        boolean isOk = Commands.preacceptInvalidate(instance, txnId, ballot);
-        SafeCommand safeCommand = instance.command(txnId);
+        boolean acceptedFastPath;

Review Comment:
   why define it here when its a simple condition in line 67?



##########
accord-core/src/main/java/accord/local/SaveStatus.java:
##########
@@ -126,21 +219,68 @@ public static SaveStatus enrich(SaveStatus status, Known 
known)
             case Accepted:
             case AcceptedInvalidate:
             case PreCommitted:
-
                 if (known.isSatisfiedBy(status.known))
                     return status;
                 return get(status.status, status.known.merge(known));
+
+            case Truncated:
+                switch (status)
+                {
+                    default: throw new AssertionError();

Review Comment:
   ```suggestion
                       default: throw new 
AssertionError(String.format("Unexpected status %s: known was %s", status, 
known));
   ```



##########
accord-core/src/main/java/accord/utils/DeterministicSet.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 accord.utils;
+
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+import com.google.common.collect.Iterables;
+
+public class DeterministicSet<T> extends AbstractSet<T>
+{
+    static class Entry<T>
+    {
+        final T item;
+        Entry<T> prev;
+        Entry<T> next;
+
+        Entry(T item)
+        {
+            this.item = item;
+        }
+    }
+
+    // TODO (low priority): an identity hash map that doesn't mind concurrent 
modification / iteration

Review Comment:
   why is this needed?  We don't `iterate` on the lookup so there isn't a 
concurrent modification problem



##########
accord-core/src/main/java/accord/utils/DeterministicSet.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 accord.utils;
+
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+import com.google.common.collect.Iterables;
+
+public class DeterministicSet<T> extends AbstractSet<T>

Review Comment:
   Can you add tests for this class?  I wrote them and think that deleting 
before iterating has an issue
   
   ```
   /*
    * 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 accord.utils;
   
   import org.junit.jupiter.api.Test;
   
   import java.util.Iterator;
   
   import static org.assertj.core.api.Assertions.assertThat;
   
   class DeterministicSetTest {
       @Test
       public void removePrevious()
       {
           DeterministicSet<Integer> set = of(1, 2, 3);
           Iterator<Integer> it = set.iterator();
           assertThat(next(it)).isEqualTo(3);
           set.remove(3);
           assertThat(next(it)).isEqualTo(2);
           assertThat(next(it)).isEqualTo(1);
           assertThat(it).isExhausted();
       }
   
       @Test
       public void removeCurrent()
       {
           DeterministicSet<Integer> set = of(1, 2, 3);
           Iterator<Integer> it = set.iterator();
           assertThat(next(it)).isEqualTo(3);
           set.remove(2);
           assertThat(next(it)).isEqualTo(1);
           assertThat(it).isExhausted();
       }
   
       @Test
       public void removeNext()
       {
           DeterministicSet<Integer> set = of(1, 2, 3);
           Iterator<Integer> it = set.iterator();
           assertThat(next(it)).isEqualTo(3);
           set.remove(1);
           assertThat(next(it)).isEqualTo(2);
           assertThat(it).isExhausted();
       }
   
       @Test
       public void removeFirst()
       {
           DeterministicSet<Integer> set = of(1, 2, 3);
           Iterator<Integer> it = set.iterator();
           set.remove(3);
           assertThat(next(it)).isEqualTo(2);
           assertThat(next(it)).isEqualTo(1);
           assertThat(it).isExhausted();
       }
   
       private static <T> DeterministicSet<T> of(T... values)
       {
           DeterministicSet<T> set = new DeterministicSet<>();
           for (T t : values)
               set.add(t);
           return set;
       }
   
       private static <T> T next(Iterator<T> it)
       {
           assertThat(it).hasNext();
           return it.next();
       }
   }
   ```
   
   In `accord.utils.DeterministicSet#iterator` I did (lazy define `next` so we 
can see the first element removed)
   
   ```
   public Iterator<T> iterator()
       {
           return new Iterator<T>()
           {
               boolean hasComputedNext = false;
               Entry<T> next = null;
               @Override
               public boolean hasNext()
               {
                   if (!hasComputedNext)
                   {
                       if (next == null)
                       {
                           next = head.next;
                           hasComputedNext = true;
                           return true;
                       }
                       // apply any deletion before deciding if hasNext
                       while (next.next == null)
                           next = next.prev;
                       next = next.next;
                       hasComputedNext = true;
                   }
                   return next != head;
               }
   
               @Override
               public T next()
               {
                   if (!hasNext())
                       throw new NoSuchElementException();
                   T result = next.item;
                   // defer filtering of deleted items until hasNext(), so 
processing of next() can have applied
                   hasComputedNext = false;
                   return result;
               }
           };
       }
   ```



##########
accord-core/src/main/java/accord/local/SaveStatus.java:
##########
@@ -126,21 +219,68 @@ public static SaveStatus enrich(SaveStatus status, Known 
known)
             case Accepted:
             case AcceptedInvalidate:
             case PreCommitted:
-
                 if (known.isSatisfiedBy(status.known))
                     return status;
                 return get(status.status, status.known.merge(known));
+
+            case Truncated:
+                switch (status)
+                {
+                    default: throw new AssertionError();
+                    case Erased:
+                        if (!known.outcome.isOrWasApply() || known.executeAt 
!= ExecuteAtKnown)
+                            return Erased;
+
+                    case TruncatedApply:
+                        if (known.outcome != Outcome.Apply)
+                            return TruncatedApply;
+
+                    case TruncatedApplyWithOutcome:
+                        if (known.deps != DepsKnown)
+                            return TruncatedApplyWithOutcome;
+
+                    case TruncatedApplyWithDeps:
+                        if (!known.isDefinitionKnown())
+                            return TruncatedApplyWithDeps;
+
+                        return Applied;
+                }
         }
 
         return status;
     }
 
-    public static SaveStatus merge(SaveStatus a, Ballot acceptedA, SaveStatus 
b, Ballot acceptedB)
+    public static SaveStatus merge(SaveStatus a, Ballot acceptedA, SaveStatus 
b, Ballot acceptedB, boolean preferKnowledge)

Review Comment:
   if I am reading this right, we will have the following
   
   ```
   merge(Erased, ..., AcceptedWithDefinition, ...) == Erased // same with 
Accepted
   ```
   
   this is due to the fact `Erased > AcceptedWithDefinition` and we use the 
`AcceptedWithDefinition.known`, so `ExecuteAtProposed != ExecuteAtKnown == true`
   
   this is done during check status, so one replica hasn't applied but has 
accepted, and the others have erased
   
   now, this "shouldn't" happen as we only define `Erased` when `if 
(commandStore().durableBefore().isUniversal(txnId, unseekable))`... 
   
   have to think through this (enrich) more...
   
   `universal` is controlled by ExclusiveSyncPoint but updated by 
`accord.coordinate.CoordinateShardDurable#coordinate`, which blocks waiting on 
`rf` replies before sending the `SetShardDurable` message... so we *should* not 
get to this case... just something that looks possible in the API....



##########
accord-core/src/main/java/accord/primitives/AbstractRanges.java:
##########
@@ -482,6 +556,31 @@ else if (c > 0)
     @Override
     public String toString()

Review Comment:
   Can you add the following test?  This method doesn't work, so the test shows 
it (and the test forces formatting to get fixed as well to be easier to read)
   
   ```
   /*
    * 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 accord.primitives;
   
   import accord.impl.IntKey;
   import org.junit.jupiter.api.Test;
   
   import javax.annotation.Nonnull;
   
   import static org.assertj.core.api.Assertions.assertThat;
   
   
   class AbstractRangesTest {
   
       @Test
       public void testToString()
       {
           Ranges ranges = new Ranges(
                   range("first", 0, 10),
                   range("first", 10, 20),
                   range("second", 20, 30),
                   range("third", 30, 40)
           );
           assertThat(ranges.toString()).isEqualTo("[first:[[0,10), [10,20)], 
second:[[20,30)], third:[[30,40)]]");
       }
   
       private static Range range(Object prefix, int start, int end)
       {
           return Range.range(new PrefixKey(start, prefix), new PrefixKey(end, 
prefix), true , false);
       }
   
       private static class Ranges extends AbstractRanges
       {
           Ranges(@Nonnull Range... ranges) {
               super(ranges);
           }
   
           @Override
           public Routables<?> slice(accord.primitives.Ranges ranges) {
               throw new UnsupportedOperationException();
           }
       }
   
       private static class PrefixKey extends IntKey.Routing
       {
           private final Object prefix;
   
           public PrefixKey(int key, Object prefix) {
               super(key);
               this.prefix = prefix;
           }
   
           @Override
           public Object prefix() {
               return prefix;
           }
       }
   }
   ```



##########
accord-core/src/test/java/accord/impl/TopologyUtils.java:
##########
@@ -64,8 +64,15 @@ public static Topology initialTopology(Node.Id[] cluster, 
Ranges ranges, int rf)
         }
 
         final List<Shard> shards = new ArrayList<>();
+        Set<Node.Id> noShard = new HashSet<>(Arrays.asList(cluster));
         for (int i = 0 ; i < ranges.size() ; ++i)
+        {
             shards.add(new Shard(ranges.get(i), electorates.get(i % 
electorates.size()), fastPathElectorates.get(i % fastPathElectorates.size())));
+            noShard.removeAll(electorates.get(i % electorates.size()));
+        }
+        if (!noShard.isEmpty())
+            throw new AssertionError();

Review Comment:
   ```suggestion
               throw new AssertionError(String.format("The following 
electorates were found without a shard: %s", noShard));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to