aweisberg commented on code in PR #4118: URL: https://github.com/apache/cassandra/pull/4118#discussion_r2086857011
########## src/java/org/apache/cassandra/db/CoordinatorLogBoundariesMap.java: ########## @@ -0,0 +1,80 @@ +/* + * 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.db; + +import java.util.Iterator; + +import javax.annotation.concurrent.ThreadSafe; + +import com.google.common.collect.Iterators; + +import org.apache.cassandra.replication.MutationId; +import org.apache.cassandra.replication.ShortMutationId; +import org.jctools.maps.NonBlockingHashMapLong; + +/** + * 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. + * Consider wrapping value in AtomicReference to avoid false sharing, see: https://trishagee.com/2011/07/22/dissecting_the_disruptor_why_its_so_fast_part_two__magic_cache_line_padding/ + */ +@ThreadSafe +public class CoordinatorLogBoundariesMap extends NonBlockingHashMapLong<MutationId> implements MutableCoordinatorLogBoundaries Review Comment: This can be package private. ########## src/java/org/apache/cassandra/db/MutableCoordinatorLogBoundaries.java: ########## @@ -0,0 +1,111 @@ +/* + * 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.db; + +import java.util.Iterator; +import java.util.Objects; + +import javax.annotation.concurrent.ThreadSafe; + +import org.apache.cassandra.replication.MutationId; +import org.apache.cassandra.replication.ShortMutationId; +import org.jctools.maps.NonBlockingHashMapLong; + +@ThreadSafe +public class MutableCoordinatorLogBoundaries extends CoordinatorLogBoundaries +{ + private static final MutationId NONE = MutationId.none(); + private static final int NONE_OFFSET = NONE.offset(); + + // 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 + private final NonBlockingHashMapLong<MutationId> ids = new NonBlockingHashMapLong<>(); + + public void add(MutationId mutationId) + { + long logId = mutationId.logId(); + ids.merge(logId, mutationId, (existing, updating) -> { + if (ShortMutationId.comparator.compare(existing, updating) < 0) + return updating; + return existing; + }); + } + + @Override + public int maxOffset(long logId) + { + MutationId id = ids.get(logId); + return id == null ? NONE_OFFSET : id.offset(); + } + + @Override + protected MutationId max(long logId) + { + return ids.getOrDefault(logId, NONE); + } + + @Override + protected int size() + { + return ids.size(); + } + + @Override + public Iterator<Long> iterator() Review Comment: Guava and Apache Commons both have helpers for creating unmodifiable iterators. ########## src/java/org/apache/cassandra/db/CoordinatorLogBoundaries.java: ########## @@ -0,0 +1,155 @@ +/* + * 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.db; + +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.sstable.metadata.StatsMetadata; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.replication.CoordinatorLogId; +import org.apache.cassandra.replication.MutationId; +import org.apache.cassandra.utils.vint.VIntCoding; + +/** + * Max mutation ID present in this SSTable for each coordinator log, to determine whether an SSTable is reconciled or + * not. Once max mutation IDs are reconciled, next compaction can safely mark this SSTabled as repaired. Note that peers + * may have reconciled all mutations included in an SSTable, but {@link StatsMetadata#repairedAt} is dependent on + * compaction timing, so "nodetool repair --validate" may report temporary disagreements on the repaired set. + * <p> + * Iterable over {@link CoordinatorLogId}. + */ +public abstract class CoordinatorLogBoundaries implements Iterable<Long> +{ + public static class Builder + { + private final MutableCoordinatorLogBoundaries boundaries = new MutableCoordinatorLogBoundaries(); + + public CoordinatorLogBoundaries build() + { + return boundaries; + } + + public void add(MutationId id) + { + boundaries.add(id); + } + + public void addAll(CoordinatorLogBoundaries from) + { + for (long logId : from) + { + MutationId max = from.max(logId); + if (!max.isNone()) + boundaries.add(max); + } + } + } + + public static Builder builder() + { + return new Builder(); + } + + public abstract int maxOffset(long logId); + protected abstract MutationId max(long logId); + protected abstract int size(); + + public static final IVersionedSerializer<CoordinatorLogBoundaries> serializer = new IVersionedSerializer<>() + { + @Override + public void serialize(CoordinatorLogBoundaries boundaries, DataOutputPlus out, int version) throws IOException + { + if (version < MessagingService.VERSION_52) + return; + out.writeUnsignedVInt32(boundaries.size()); + for (long logId : boundaries) + MutationId.serializer.serialize(boundaries.max(logId), out, version); + } + + @Override + public CoordinatorLogBoundaries deserialize(DataInputPlus in, int version) throws IOException + { + int size = in.readUnsignedVInt32(); + Builder builder = CoordinatorLogBoundaries.builder(); + for (int i = 0; i < size; i++) + { + MutationId mutationId = MutationId.serializer.deserialize(in, version); + builder.add(mutationId); + } + return builder.build(); + } + + @Override + public long serializedSize(CoordinatorLogBoundaries boundaries, int version) + { + if (version < MessagingService.VERSION_52) + return 0; + long size = 0; + size += VIntCoding.computeUnsignedVIntSize(boundaries.size()); + for (long logId : boundaries) + size += MutationId.serializer.serializedSize(boundaries.max(logId), version); + return size; + } + }; + + public static final CoordinatorLogBoundaries NONE = new CoordinatorLogBoundaries() Review Comment: We do this constantly so I wouldn't worry about it. If it deadlocked many different things would have to change. ########## src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java: ########## @@ -123,17 +124,10 @@ public Map<MetadataType, MetadataComponent> deserialize(Descriptor descriptor, E logger.trace("Load metadata for {}", descriptor); File statsFile = descriptor.fileFor(Components.STATS); if (!statsFile.exists()) + throw new NoSuchFileException("Stats component of sstable " + descriptor + " is missing"); Review Comment: If it's for 2.1 upgrade I think it's safe to remove at this 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: pr-unsubscr...@cassandra.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org For additional commands, e-mail: pr-h...@cassandra.apache.org