ifesdjeen commented on code in PR #3408: URL: https://github.com/apache/cassandra/pull/3408#discussion_r1674661593
########## test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingSemaphore.java: ########## @@ -0,0 +1,180 @@ +/* + * 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.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.permits(); + + int current = permits.get(); + boolean res = permits.compareAndSet(current, 0); + assert res; + return current; + } + + @Override + public void release(int release) + { + if (ifIntercepted() == null) + { + super.release(); + return; + } + + int current = permits.get(); + permits.compareAndSet(current, current + release); + int remaining = permits.get(); + 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); + + int current = permits.get(); + if (current >= acquire) + { + return permits.compareAndSet(current, current - acquire); + } + else + { + return false; + } + } + + @Override + public boolean tryAcquire(int acquire, long time, TimeUnit unit) throws InterruptedException + { + if (ifIntercepted() == null) + return super.tryAcquire(acquire); + + int current = permits.get(); + if (current >= acquire) + return permits.compareAndSet(current, current - acquire); + + SemaphoreSignal signal = new SemaphoreSignal(acquire); + interceptible.add(signal); + signal.await(time, unit); + return permits.compareAndSet(current, current - acquire); + } + + @Override + public boolean tryAcquireUntil(int acquire, long nanoTimeDeadline) throws InterruptedException + { + if (ifIntercepted() == null) + return super.tryAcquireUntil(acquire, nanoTimeDeadline); + + int current = permits.get(); + if (current >= acquire) + return permits.compareAndSet(current, current - acquire); Review Comment: switched to add a loop ########## test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java: ########## @@ -18,250 +18,359 @@ package org.apache.cassandra.simulator.test; import java.io.IOException; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.CopyOnWriteArrayList; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Optional; +import java.util.Set; +import java.util.zip.Checksum; import javax.annotation.Nullable; import com.google.common.collect.ImmutableMap; +import com.google.common.jimfs.Jimfs; -import accord.topology.TopologyUtils; +import org.apache.cassandra.concurrent.ExecutorFactory; +import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.AccordSpec; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.distributed.test.log.CMSTestBase; +import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.io.filesystem.ListenableFileSystem; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.journal.AsyncCallbacks; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.journal.KeySupport; +import org.apache.cassandra.journal.ValueSerializer; import org.apache.cassandra.schema.*; -import org.junit.Ignore; + +import org.junit.Assert; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.Utils; -import accord.api.Data; -import accord.api.RoutingKey; -import accord.api.Update; -import accord.api.Write; -import accord.local.Node; -import accord.messages.MessageType; -import accord.messages.PreAccept; -import accord.messages.TxnRequest; -import accord.primitives.FullKeyRoute; -import accord.primitives.FullRoute; -import accord.primitives.Keys; -import accord.primitives.Ranges; -import accord.primitives.Seekables; -import accord.primitives.Timestamp; -import accord.primitives.Txn; -import accord.primitives.TxnId; -import accord.topology.Topologies; -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.ExecutorPlus; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.ParameterizedClass; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.commitlog.CommitLog; -import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.Files; -import org.apache.cassandra.service.accord.AccordJournal; -import org.apache.cassandra.service.accord.TokenRange; -import org.apache.cassandra.service.accord.api.AccordRoutingKey; -import org.apache.cassandra.service.accord.api.PartitionKey; -import org.apache.cassandra.service.accord.txn.TxnNamedRead; -import org.apache.cassandra.service.accord.txn.TxnQuery; -import org.apache.cassandra.service.accord.txn.TxnRead; -import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.tcm.AtomicLongBackedProcessor; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Isolated; import org.apache.cassandra.utils.concurrent.CountDownLatch; public class AccordJournalSimulationTest extends SimulationTestBase { @Test - @Ignore // TODO: re-enable - public void test() throws IOException + public void simpleRWTest() throws IOException { - simulate(arr(() -> run()), - () -> check()); + simulate(arr(() -> { + ListenableFileSystem fs = new ListenableFileSystem(Jimfs.newFileSystem()); + File.unsafeSetFilesystem(fs); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setCommitLogCompression(new ParameterizedClass("LZ4Compressor", ImmutableMap.of())); // + DatabaseDescriptor.setCommitLogWriteDiskAccessMode(Config.DiskAccessMode.standard); + DatabaseDescriptor.initializeCommitLogDiskAccessMode(); + System.out.println("DatabaseDescriptor.getCommitLogWriteDiskAccessMode() = " + DatabaseDescriptor.getCommitLogWriteDiskAccessMode()); + + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + long seed = 1L; + DatabaseDescriptor.setAccordJournalDirectory("/journal"); + new File("/journal").createDirectoriesIfNotExists(); + + DatabaseDescriptor.setDumpHeapOnUncaughtException(false); + + Keyspace.setInitialized(); + + CMSTestBase.CMSSut sut = new CMSTestBase.CMSSut(AtomicLongBackedProcessor::new, + false, + new FakeSchema(), + new TokenPlacementModel.SimpleReplicationFactor(1)); + + State.journal = new Journal<>("AccordJournal", + new File("/journal"), + new AccordSpec.JournalSpec(), + new TestCallbacks(), + new IdentityKeySerializer(), + new IdentityValueSerializer()); + }), + () -> check()); } - private static void run() + public static void check() { - for (int i = 0; i < State.events; i++) + State.journal.start(); + try + { + for (int i = 0; i < 100; i++) + { + int finalI = i; + State.executor.submit(() -> State.journal.asyncWrite("test" + finalI, "test" + finalI, Collections.singleton(1), null)); + } + + State.latch.await(); + + for (int i = 0; i < 100; i++) + { + System.out.println("Reading " + State.journal.readFirst("test" + i)); Review Comment: removed; probably cleaner this way ########## test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java: ########## @@ -18,250 +18,359 @@ package org.apache.cassandra.simulator.test; import java.io.IOException; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.CopyOnWriteArrayList; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Optional; +import java.util.Set; +import java.util.zip.Checksum; import javax.annotation.Nullable; import com.google.common.collect.ImmutableMap; +import com.google.common.jimfs.Jimfs; -import accord.topology.TopologyUtils; +import org.apache.cassandra.concurrent.ExecutorFactory; +import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.AccordSpec; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.distributed.test.log.CMSTestBase; +import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.io.filesystem.ListenableFileSystem; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.journal.AsyncCallbacks; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.journal.KeySupport; +import org.apache.cassandra.journal.ValueSerializer; import org.apache.cassandra.schema.*; -import org.junit.Ignore; + +import org.junit.Assert; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.Utils; -import accord.api.Data; -import accord.api.RoutingKey; -import accord.api.Update; -import accord.api.Write; -import accord.local.Node; -import accord.messages.MessageType; -import accord.messages.PreAccept; -import accord.messages.TxnRequest; -import accord.primitives.FullKeyRoute; -import accord.primitives.FullRoute; -import accord.primitives.Keys; -import accord.primitives.Ranges; -import accord.primitives.Seekables; -import accord.primitives.Timestamp; -import accord.primitives.Txn; -import accord.primitives.TxnId; -import accord.topology.Topologies; -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.ExecutorPlus; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.ParameterizedClass; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.commitlog.CommitLog; -import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.Files; -import org.apache.cassandra.service.accord.AccordJournal; -import org.apache.cassandra.service.accord.TokenRange; -import org.apache.cassandra.service.accord.api.AccordRoutingKey; -import org.apache.cassandra.service.accord.api.PartitionKey; -import org.apache.cassandra.service.accord.txn.TxnNamedRead; -import org.apache.cassandra.service.accord.txn.TxnQuery; -import org.apache.cassandra.service.accord.txn.TxnRead; -import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.tcm.AtomicLongBackedProcessor; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Isolated; import org.apache.cassandra.utils.concurrent.CountDownLatch; public class AccordJournalSimulationTest extends SimulationTestBase { @Test - @Ignore // TODO: re-enable - public void test() throws IOException + public void simpleRWTest() throws IOException { - simulate(arr(() -> run()), - () -> check()); + simulate(arr(() -> { + ListenableFileSystem fs = new ListenableFileSystem(Jimfs.newFileSystem()); + File.unsafeSetFilesystem(fs); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setCommitLogCompression(new ParameterizedClass("LZ4Compressor", ImmutableMap.of())); // + DatabaseDescriptor.setCommitLogWriteDiskAccessMode(Config.DiskAccessMode.standard); + DatabaseDescriptor.initializeCommitLogDiskAccessMode(); + System.out.println("DatabaseDescriptor.getCommitLogWriteDiskAccessMode() = " + DatabaseDescriptor.getCommitLogWriteDiskAccessMode()); + + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + long seed = 1L; + DatabaseDescriptor.setAccordJournalDirectory("/journal"); + new File("/journal").createDirectoriesIfNotExists(); + + DatabaseDescriptor.setDumpHeapOnUncaughtException(false); + + Keyspace.setInitialized(); + + CMSTestBase.CMSSut sut = new CMSTestBase.CMSSut(AtomicLongBackedProcessor::new, + false, + new FakeSchema(), + new TokenPlacementModel.SimpleReplicationFactor(1)); + + State.journal = new Journal<>("AccordJournal", + new File("/journal"), + new AccordSpec.JournalSpec(), + new TestCallbacks(), + new IdentityKeySerializer(), + new IdentityValueSerializer()); + }), + () -> check()); } - private static void run() + public static void check() { - for (int i = 0; i < State.events; i++) + State.journal.start(); + try + { + for (int i = 0; i < 100; i++) + { + int finalI = i; + State.executor.submit(() -> State.journal.asyncWrite("test" + finalI, "test" + finalI, Collections.singleton(1), null)); + } + + State.latch.await(); + + for (int i = 0; i < 100; i++) + { + System.out.println("Reading " + State.journal.readFirst("test" + i)); + Assert.assertEquals(State.journal.readFirst("test" + i), "test" + i); + } + } + catch (Throwable e) { - int finalI = i; - State.executor.execute(() -> State.append(finalI)); + e.printStackTrace(); Review Comment: switched to just collect and throw ########## test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java: ########## @@ -18,250 +18,359 @@ package org.apache.cassandra.simulator.test; import java.io.IOException; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.CopyOnWriteArrayList; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Optional; +import java.util.Set; +import java.util.zip.Checksum; import javax.annotation.Nullable; import com.google.common.collect.ImmutableMap; +import com.google.common.jimfs.Jimfs; -import accord.topology.TopologyUtils; +import org.apache.cassandra.concurrent.ExecutorFactory; +import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.AccordSpec; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.distributed.test.log.CMSTestBase; +import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.io.filesystem.ListenableFileSystem; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.journal.AsyncCallbacks; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.journal.KeySupport; +import org.apache.cassandra.journal.ValueSerializer; import org.apache.cassandra.schema.*; -import org.junit.Ignore; + +import org.junit.Assert; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.Utils; -import accord.api.Data; -import accord.api.RoutingKey; -import accord.api.Update; -import accord.api.Write; -import accord.local.Node; -import accord.messages.MessageType; -import accord.messages.PreAccept; -import accord.messages.TxnRequest; -import accord.primitives.FullKeyRoute; -import accord.primitives.FullRoute; -import accord.primitives.Keys; -import accord.primitives.Ranges; -import accord.primitives.Seekables; -import accord.primitives.Timestamp; -import accord.primitives.Txn; -import accord.primitives.TxnId; -import accord.topology.Topologies; -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.ExecutorPlus; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.ParameterizedClass; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.commitlog.CommitLog; -import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.Files; -import org.apache.cassandra.service.accord.AccordJournal; -import org.apache.cassandra.service.accord.TokenRange; -import org.apache.cassandra.service.accord.api.AccordRoutingKey; -import org.apache.cassandra.service.accord.api.PartitionKey; -import org.apache.cassandra.service.accord.txn.TxnNamedRead; -import org.apache.cassandra.service.accord.txn.TxnQuery; -import org.apache.cassandra.service.accord.txn.TxnRead; -import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.tcm.AtomicLongBackedProcessor; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Isolated; import org.apache.cassandra.utils.concurrent.CountDownLatch; public class AccordJournalSimulationTest extends SimulationTestBase { @Test - @Ignore // TODO: re-enable - public void test() throws IOException + public void simpleRWTest() throws IOException { - simulate(arr(() -> run()), - () -> check()); + simulate(arr(() -> { + ListenableFileSystem fs = new ListenableFileSystem(Jimfs.newFileSystem()); + File.unsafeSetFilesystem(fs); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setCommitLogCompression(new ParameterizedClass("LZ4Compressor", ImmutableMap.of())); // + DatabaseDescriptor.setCommitLogWriteDiskAccessMode(Config.DiskAccessMode.standard); + DatabaseDescriptor.initializeCommitLogDiskAccessMode(); + System.out.println("DatabaseDescriptor.getCommitLogWriteDiskAccessMode() = " + DatabaseDescriptor.getCommitLogWriteDiskAccessMode()); Review Comment: Good point; removed. ########## test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java: ########## @@ -18,250 +18,359 @@ package org.apache.cassandra.simulator.test; import java.io.IOException; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.CopyOnWriteArrayList; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Optional; +import java.util.Set; +import java.util.zip.Checksum; import javax.annotation.Nullable; import com.google.common.collect.ImmutableMap; +import com.google.common.jimfs.Jimfs; -import accord.topology.TopologyUtils; +import org.apache.cassandra.concurrent.ExecutorFactory; +import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.AccordSpec; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.distributed.test.log.CMSTestBase; +import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.io.filesystem.ListenableFileSystem; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.journal.AsyncCallbacks; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.journal.KeySupport; +import org.apache.cassandra.journal.ValueSerializer; import org.apache.cassandra.schema.*; -import org.junit.Ignore; + +import org.junit.Assert; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.Utils; -import accord.api.Data; -import accord.api.RoutingKey; -import accord.api.Update; -import accord.api.Write; -import accord.local.Node; -import accord.messages.MessageType; -import accord.messages.PreAccept; -import accord.messages.TxnRequest; -import accord.primitives.FullKeyRoute; -import accord.primitives.FullRoute; -import accord.primitives.Keys; -import accord.primitives.Ranges; -import accord.primitives.Seekables; -import accord.primitives.Timestamp; -import accord.primitives.Txn; -import accord.primitives.TxnId; -import accord.topology.Topologies; -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.ExecutorPlus; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.ParameterizedClass; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.commitlog.CommitLog; -import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.Files; -import org.apache.cassandra.service.accord.AccordJournal; -import org.apache.cassandra.service.accord.TokenRange; -import org.apache.cassandra.service.accord.api.AccordRoutingKey; -import org.apache.cassandra.service.accord.api.PartitionKey; -import org.apache.cassandra.service.accord.txn.TxnNamedRead; -import org.apache.cassandra.service.accord.txn.TxnQuery; -import org.apache.cassandra.service.accord.txn.TxnRead; -import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.tcm.AtomicLongBackedProcessor; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Isolated; import org.apache.cassandra.utils.concurrent.CountDownLatch; public class AccordJournalSimulationTest extends SimulationTestBase { @Test - @Ignore // TODO: re-enable - public void test() throws IOException + public void simpleRWTest() throws IOException { - simulate(arr(() -> run()), - () -> check()); + simulate(arr(() -> { + ListenableFileSystem fs = new ListenableFileSystem(Jimfs.newFileSystem()); + File.unsafeSetFilesystem(fs); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setCommitLogCompression(new ParameterizedClass("LZ4Compressor", ImmutableMap.of())); // + DatabaseDescriptor.setCommitLogWriteDiskAccessMode(Config.DiskAccessMode.standard); + DatabaseDescriptor.initializeCommitLogDiskAccessMode(); + System.out.println("DatabaseDescriptor.getCommitLogWriteDiskAccessMode() = " + DatabaseDescriptor.getCommitLogWriteDiskAccessMode()); + + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + long seed = 1L; + DatabaseDescriptor.setAccordJournalDirectory("/journal"); + new File("/journal").createDirectoriesIfNotExists(); + + DatabaseDescriptor.setDumpHeapOnUncaughtException(false); + + Keyspace.setInitialized(); + + CMSTestBase.CMSSut sut = new CMSTestBase.CMSSut(AtomicLongBackedProcessor::new, + false, + new FakeSchema(), + new TokenPlacementModel.SimpleReplicationFactor(1)); Review Comment: On closer inspection, you are right, since this is not `AccordJournal` we do not need to set up schema here! ########## test/simulator/main/org/apache/cassandra/simulator/systems/InterceptingSemaphore.java: ########## @@ -0,0 +1,180 @@ +/* + * 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.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.permits(); + + int current = permits.get(); + boolean res = permits.compareAndSet(current, 0); + assert res; + return current; + } + + @Override + public void release(int release) + { + if (ifIntercepted() == null) + { + super.release(); + return; + } + + int current = permits.get(); + permits.compareAndSet(current, current + release); + int remaining = permits.get(); + 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); + + int current = permits.get(); + if (current >= acquire) + { + return permits.compareAndSet(current, current - acquire); Review Comment: switched to add loop here ########## test/simulator/test/org/apache/cassandra/simulator/test/AccordJournalSimulationTest.java: ########## @@ -18,250 +18,359 @@ package org.apache.cassandra.simulator.test; import java.io.IOException; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.CopyOnWriteArrayList; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Optional; +import java.util.Set; +import java.util.zip.Checksum; import javax.annotation.Nullable; import com.google.common.collect.ImmutableMap; +import com.google.common.jimfs.Jimfs; -import accord.topology.TopologyUtils; +import org.apache.cassandra.concurrent.ExecutorFactory; +import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.AccordSpec; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.distributed.test.log.CMSTestBase; +import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.io.filesystem.ListenableFileSystem; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.journal.AsyncCallbacks; +import org.apache.cassandra.journal.Journal; +import org.apache.cassandra.journal.KeySupport; +import org.apache.cassandra.journal.ValueSerializer; import org.apache.cassandra.schema.*; -import org.junit.Ignore; + +import org.junit.Assert; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import accord.Utils; -import accord.api.Data; -import accord.api.RoutingKey; -import accord.api.Update; -import accord.api.Write; -import accord.local.Node; -import accord.messages.MessageType; -import accord.messages.PreAccept; -import accord.messages.TxnRequest; -import accord.primitives.FullKeyRoute; -import accord.primitives.FullRoute; -import accord.primitives.Keys; -import accord.primitives.Ranges; -import accord.primitives.Seekables; -import accord.primitives.Timestamp; -import accord.primitives.Txn; -import accord.primitives.TxnId; -import accord.topology.Topologies; -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.ExecutorPlus; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.ParameterizedClass; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.commitlog.CommitLog; -import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.Files; -import org.apache.cassandra.service.accord.AccordJournal; -import org.apache.cassandra.service.accord.TokenRange; -import org.apache.cassandra.service.accord.api.AccordRoutingKey; -import org.apache.cassandra.service.accord.api.PartitionKey; -import org.apache.cassandra.service.accord.txn.TxnNamedRead; -import org.apache.cassandra.service.accord.txn.TxnQuery; -import org.apache.cassandra.service.accord.txn.TxnRead; -import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.tcm.AtomicLongBackedProcessor; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Isolated; import org.apache.cassandra.utils.concurrent.CountDownLatch; public class AccordJournalSimulationTest extends SimulationTestBase { @Test - @Ignore // TODO: re-enable - public void test() throws IOException + public void simpleRWTest() throws IOException { - simulate(arr(() -> run()), - () -> check()); + simulate(arr(() -> { + ListenableFileSystem fs = new ListenableFileSystem(Jimfs.newFileSystem()); + File.unsafeSetFilesystem(fs); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setCommitLogCompression(new ParameterizedClass("LZ4Compressor", ImmutableMap.of())); // + DatabaseDescriptor.setCommitLogWriteDiskAccessMode(Config.DiskAccessMode.standard); + DatabaseDescriptor.initializeCommitLogDiskAccessMode(); + System.out.println("DatabaseDescriptor.getCommitLogWriteDiskAccessMode() = " + DatabaseDescriptor.getCommitLogWriteDiskAccessMode()); + + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + long seed = 1L; + DatabaseDescriptor.setAccordJournalDirectory("/journal"); + new File("/journal").createDirectoriesIfNotExists(); + + DatabaseDescriptor.setDumpHeapOnUncaughtException(false); + + Keyspace.setInitialized(); + + CMSTestBase.CMSSut sut = new CMSTestBase.CMSSut(AtomicLongBackedProcessor::new, + false, + new FakeSchema(), + new TokenPlacementModel.SimpleReplicationFactor(1)); + + State.journal = new Journal<>("AccordJournal", + new File("/journal"), + new AccordSpec.JournalSpec(), + new TestCallbacks(), + new IdentityKeySerializer(), + new IdentityValueSerializer()); + }), + () -> check()); } - private static void run() + public static void check() { - for (int i = 0; i < State.events; i++) + State.journal.start(); + try + { + for (int i = 0; i < 100; i++) + { + int finalI = i; + State.executor.submit(() -> State.journal.asyncWrite("test" + finalI, "test" + finalI, Collections.singleton(1), null)); + } + + State.latch.await(); + + for (int i = 0; i < 100; i++) + { + System.out.println("Reading " + State.journal.readFirst("test" + i)); + Assert.assertEquals(State.journal.readFirst("test" + i), "test" + i); + } + } + catch (Throwable e) { - int finalI = i; - State.executor.execute(() -> State.append(finalI)); + e.printStackTrace(); } + finally + { + State.journal.shutdown(); + } + } - try + public static class TestCallbacks implements AsyncCallbacks<String, String> + { + + @Override + public void onWrite(long segment, int position, int size, String key, String value, Object writeContext) { - State.eventsDurable.await(); - State.logger.info("All events are durable done!"); + State.latch.decrement(); } - catch (InterruptedException e) + + @Override + public void onWriteFailed(String key, String value, Object writeContext, Throwable cause) { - throw new AssertionError(e); + State.latch.decrement(); Review Comment: good point! -- 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]

