aratno commented on code in PR #4192: URL: https://github.com/apache/cassandra/pull/4192#discussion_r2164996626
########## src/java/org/apache/cassandra/replication/CoordinatorLogOffsetsMap.java: ########## @@ -0,0 +1,205 @@ +/* + * 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.replication; + +import java.util.Iterator; +import java.util.Objects; +import java.util.concurrent.locks.ReentrantLock; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.db.memtable.SkipListMemtable; +import org.apache.cassandra.db.memtable.TrieMemtable; +import org.jctools.maps.NonBlockingHashMapLong; +import org.jctools.queues.MpscUnboundedArrayQueue; + +/** + * A replica can only receive writes from another replica it shares ranges with, and tracked writes are executed by + * coordinators, so this should contain up to (2*RF - 1) keys. + * <p> + * This is different from {@link Log2OffsetsMap} because it's focused on supporting fast, frequent updates from multiple + * threads at {@link Memtable#put}, and infrequent reads at {@link Memtable#getFlushSet}. + */ +@ThreadSafe +class CoordinatorLogOffsetsMap extends NonBlockingHashMapLong<CoordinatorLogOffsetsMap.Entry> implements MutableCoordinatorLogOffsets +{ + /** + * 0 if no contentions. + */ + private final int contentions; + + protected CoordinatorLogOffsetsMap(boolean assumeExclusive) + { + super(); + if (assumeExclusive) + this.contentions = 0; + else + this.contentions = DatabaseDescriptor.getConcurrentWriters(); + } + + public CoordinatorLogOffsetsMap() + { + super(); + this.contentions = 0; + } + + static class Entry + { + /** + * {@link SkipListMemtable} requires a lock because otherwise it uses atomics, but {@link TrieMemtable} does not, + * because each shard is already protected by a lock. In the contended case, don't wait for lock acquisition on + * update. + */ + @Nullable private final ReentrantLock lock; + @Nullable private final MpscUnboundedArrayQueue<Integer> contended; + + private final Offsets.Mutable base; + + private Entry(CoordinatorLogId logId, int contentions) + { + this.base = new Offsets.Mutable(logId); + + if (contentions == 0) + { + this.lock = null; + this.contended = null; + } + else + { + this.lock = new ReentrantLock(); + this.contended = new MpscUnboundedArrayQueue<>(contentions); Review Comment: This ends up being the initial capacity of the queue: https://github.com/JCTools/JCTools/blob/a17b56f09cc9ec7d773707578c89f3ca9c11f968/jctools-core/src/main/java/org/jctools/queues/MpscUnboundedArrayQueue.java#L19 My thinking is that we should limit resizes to avoid fragmentation, and the number of live instances of these contention queues should be limited to the number of memtables. I avoided using the bounded Mpsc implementations because those could block writes in the case of an operator dynamically increasing concurrent_writes. -- 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]

