dcapwell commented on code in PR #3408: URL: https://github.com/apache/cassandra/pull/3408#discussion_r1674720471
########## test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingSemaphore.java: ########## @@ -0,0 +1,187 @@ +/* + * 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.cassandra.simulator.systems; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.concurrent.Semaphore; +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; + +import static org.apache.cassandra.simulator.systems.InterceptorOfGlobalMethods.Global.ifIntercepted; + +public class InterceptingSemaphore extends Semaphore.Standard +{ + final Queue<SemaphoreSignal> interceptible = new ConcurrentLinkedQueue<>(); + final AtomicInteger permits; + final boolean fair; + + private static class SemaphoreSignal extends InterceptingAwaitable.InterceptingSignal<Void> + { + private final int permits; + + private SemaphoreSignal(int permits) + { + this.permits = permits; + } + } + + public InterceptingSemaphore(int permits, boolean fair) + { + super(permits); + this.permits = new AtomicInteger(permits); + this.fair = fair; + } + + @Override + public int permits() + { + if (ifIntercepted() == null) + return super.permits(); + + return permits.get(); + } + + @Override + public int drain() + { + if (ifIntercepted() == null) + return super.drain(); + + for (int i = 0; i < 10; i++) + { + int current = permits.get(); + if (permits.compareAndSet(current, 0)) + return current; + } + + throw new IllegalStateException("Too much contention"); + } + + @Override + public void release(int release) + { + if (ifIntercepted() == null) + { + super.release(release); + return; + } + + int remaining = permits.addAndGet(release); + while (!interceptible.isEmpty() && remaining > 0) + { + SemaphoreSignal signal = interceptible.peek(); + if (signal.permits >= remaining) + interceptible.poll().signal(); + else if (fair) + // Do not break enqueue order if using fair scheduler + break; + } + } + + @Override + public boolean tryAcquire(int acquire) + { + if (ifIntercepted() == null) + return super.tryAcquire(acquire); + + for (int i = 0; i < 10; i++) + { + int current = permits.get(); + if (current >= acquire) + { + if (permits.compareAndSet(current, current - acquire)) + return true; + } + else + { + return false; + } + } + throw new IllegalStateException("Too much contention"); + } + + @Override + public boolean tryAcquire(int acquire, long time, TimeUnit unit) throws InterruptedException + { + if (ifIntercepted() == null) + return super.tryAcquire(acquire, time, unit); + + return tryAcquireUntil(acquire, Clock.Global.nanoTime() + unit.toNanos(time)); Review Comment: `Clock.Global.nanoTime()` is this right in the context of Simulator? This feels wrong to me... ########## test/simulator/test/org/apache/cassandra/simulator/test/TrivialSimulationTest.java: ########## @@ -85,9 +97,97 @@ public void componentTest() } @Test - public void identityHashMapTest() + public void semaphoreTest() { - simulate(arr(() -> new IdentityHashMap<>().put(1, 1)), - () -> {}); + long seed = System.currentTimeMillis(); + semaphoreTestInternal(seed); + State.record = false; + // Verify that subsequent interleavings will be the same + semaphoreTestInternal(seed); + } + + protected void semaphoreTestInternal(long seed) + { + simulate(arr(() -> { + ExecutorPlus executor = ExecutorFactory.Global.executorFactory().pooled("semaphore-test-", 10); + Semaphore semaphore = Semaphore.newSemaphore(5); + CountDownLatch latch = CountDownLatch.newCountDownLatch(5); + + for (int i = 0; i < 5; i++) + { + int thread = i; + executor.submit(() -> { + for (int j = 0; j < 100; j++) + { + int permits = semaphore.permits(); + Assert.assertTrue(permits + " should be non negative", permits >= 0); + + try + { + semaphore.acquire(1); + State.tick(thread, j); + semaphore.release(1); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + } + latch.decrement(); + }); + } + + latch.awaitUninterruptibly(); + int permits = semaphore.permits(); + Assert.assertEquals(5, permits); + }), + () -> {}, + seed); + } + + @Shared + public static class State + { + static final List<Tick> ticks = new ArrayList<>(); + static boolean record = true; + static int i = 0; + + public static void tick(int thread, int iteration) Review Comment: is this thread safe? I don't think so ########## test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingSemaphore.java: ########## @@ -0,0 +1,187 @@ +/* + * 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.cassandra.simulator.systems; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.concurrent.Semaphore; +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; + +import static org.apache.cassandra.simulator.systems.InterceptorOfGlobalMethods.Global.ifIntercepted; + +public class InterceptingSemaphore extends Semaphore.Standard +{ + final Queue<SemaphoreSignal> interceptible = new ConcurrentLinkedQueue<>(); + final AtomicInteger permits; + final boolean fair; + + private static class SemaphoreSignal extends InterceptingAwaitable.InterceptingSignal<Void> + { + private final int permits; + + private SemaphoreSignal(int permits) + { + this.permits = permits; + } + } + + public InterceptingSemaphore(int permits, boolean fair) + { + super(permits); + this.permits = new AtomicInteger(permits); + this.fair = fair; + } + + @Override + public int permits() + { + if (ifIntercepted() == null) + return super.permits(); + + return permits.get(); + } + + @Override + public int drain() + { + if (ifIntercepted() == null) + return super.drain(); + + for (int i = 0; i < 10; i++) + { + int current = permits.get(); + if (permits.compareAndSet(current, 0)) + return current; + } + + throw new IllegalStateException("Too much contention"); + } + + @Override + public void release(int release) + { + if (ifIntercepted() == null) + { + super.release(release); + return; + } + + int remaining = permits.addAndGet(release); + while (!interceptible.isEmpty() && remaining > 0) + { + SemaphoreSignal signal = interceptible.peek(); + if (signal.permits >= remaining) + interceptible.poll().signal(); + else if (fair) + // Do not break enqueue order if using fair scheduler + break; + } + } + + @Override + public boolean tryAcquire(int acquire) + { + if (ifIntercepted() == null) + return super.tryAcquire(acquire); + + for (int i = 0; i < 10; i++) + { + int current = permits.get(); + if (current >= acquire) + { + if (permits.compareAndSet(current, current - acquire)) + return true; + } + else + { + return false; + } + } + throw new IllegalStateException("Too much contention"); + } + + @Override + public boolean tryAcquire(int acquire, long time, TimeUnit unit) throws InterruptedException + { + if (ifIntercepted() == null) + return super.tryAcquire(acquire, time, unit); + + return tryAcquireUntil(acquire, Clock.Global.nanoTime() + unit.toNanos(time)); + } + + @Override + public boolean tryAcquireUntil(int acquire, long deadline) throws InterruptedException + { + if (ifIntercepted() == null) + return super.tryAcquireUntil(acquire, deadline); + + do + { + int current = permits.get(); + if (current >= acquire) + { + if (permits.compareAndSet(current, current - acquire)) + return true; + } + SemaphoreSignal signal = new SemaphoreSignal(acquire); + interceptible.add(signal); + signal.awaitUntil(deadline); + } + while (Clock.Global.nanoTime() < deadline); Review Comment: if we waited until the deadline, then check again and not able to get the lock... shouldn't we stop right away? If we loop again are we not doing it wrong? ########## test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingSemaphore.java: ########## @@ -0,0 +1,187 @@ +/* + * 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.cassandra.simulator.systems; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.concurrent.Semaphore; +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; + +import static org.apache.cassandra.simulator.systems.InterceptorOfGlobalMethods.Global.ifIntercepted; + +public class InterceptingSemaphore extends Semaphore.Standard +{ + final Queue<SemaphoreSignal> interceptible = new ConcurrentLinkedQueue<>(); + final AtomicInteger permits; + final boolean fair; + + private static class SemaphoreSignal extends InterceptingAwaitable.InterceptingSignal<Void> + { + private final int permits; + + private SemaphoreSignal(int permits) + { + this.permits = permits; + } + } + + public InterceptingSemaphore(int permits, boolean fair) + { + super(permits); + this.permits = new AtomicInteger(permits); + this.fair = fair; + } + + @Override + public int permits() + { + if (ifIntercepted() == null) + return super.permits(); + + return permits.get(); + } + + @Override + public int drain() + { + if (ifIntercepted() == null) + return super.drain(); + + for (int i = 0; i < 10; i++) + { + int current = permits.get(); + if (permits.compareAndSet(current, 0)) + return current; + } + + throw new IllegalStateException("Too much contention"); + } + + @Override + public void release(int release) + { + if (ifIntercepted() == null) + { + super.release(release); + return; + } + + int remaining = permits.addAndGet(release); + while (!interceptible.isEmpty() && remaining > 0) + { + SemaphoreSignal signal = interceptible.peek(); + if (signal.permits >= remaining) + interceptible.poll().signal(); + else if (fair) + // Do not break enqueue order if using fair scheduler + break; + } + } + + @Override + public boolean tryAcquire(int acquire) + { + if (ifIntercepted() == null) + return super.tryAcquire(acquire); + + for (int i = 0; i < 10; i++) + { + int current = permits.get(); + if (current >= acquire) + { + if (permits.compareAndSet(current, current - acquire)) + return true; + } + else + { + return false; + } + } + throw new IllegalStateException("Too much contention"); + } + + @Override + public boolean tryAcquire(int acquire, long time, TimeUnit unit) throws InterruptedException + { + if (ifIntercepted() == null) + return super.tryAcquire(acquire, time, unit); + + return tryAcquireUntil(acquire, Clock.Global.nanoTime() + unit.toNanos(time)); + } + + @Override + public boolean tryAcquireUntil(int acquire, long deadline) throws InterruptedException + { + if (ifIntercepted() == null) + return super.tryAcquireUntil(acquire, deadline); + + do + { + int current = permits.get(); + if (current >= acquire) + { + if (permits.compareAndSet(current, current - acquire)) + return true; + } + SemaphoreSignal signal = new SemaphoreSignal(acquire); + interceptible.add(signal); + signal.awaitUntil(deadline); + } + while (Clock.Global.nanoTime() < deadline); Review Comment: `Clock.Global.nanoTime()` feels wrong as time is owned by Simulator, so the logical time could be multiple hours but the physical time is just a few ms, which would lead to this doing the wrong behavior (and being non-deterministic)? -- 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]

