dcapwell commented on code in PR #2256: URL: https://github.com/apache/cassandra/pull/2256#discussion_r1160293780
########## src/java/org/apache/cassandra/journal/Params.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.journal; + +public interface Params +{ + enum FlushMode { BATCH, GROUP, PERIODIC } + + enum FailurePolicy { STOP, STOP_JOURNAL, IGNORE, DIE } + + /** + * @return maximum segment size + */ + int segmentSize(); + + /** + * @return this journal's {@link FailurePolicy} + */ + FailurePolicy failurePolicy(); + + /** + * @return journal flush (sync) mode + */ + FlushMode flushMode(); + + /** + * @return milliseconds between journal flushes + */ + int flushPeriod(); + + /** + * @return milliseconds to block writes for while waiting for a slow disk flush to complete + * when in {@link FlushMode#PERIODIC} mode + */ + int periodicFlushLagBlock(); + + /** + * @return user provided version to use for key and value serialization + */ + int userVersion(); Review Comment: my last comment was to expose this via yaml, but we wouldn't want this as an `int` there, we would want an `enum` (allow opt-into new versions)... guess we could use `@JsonIgnore` to hid it... guess `Params` shouldn't be yaml, but likely `DEFAULT` would, where it takes an enum and maps that to this `int` ########## src/java/org/apache/cassandra/journal/Params.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.journal; + +public interface Params +{ + enum FlushMode { BATCH, GROUP, PERIODIC } + + enum FailurePolicy { STOP, STOP_JOURNAL, IGNORE, DIE } + + /** + * @return maximum segment size + */ + int segmentSize(); + + /** + * @return this journal's {@link FailurePolicy} + */ + FailurePolicy failurePolicy(); + + /** + * @return journal flush (sync) mode + */ + FlushMode flushMode(); + + /** + * @return milliseconds between journal flushes + */ + int flushPeriod(); + + /** + * @return milliseconds to block writes for while waiting for a slow disk flush to complete + * when in {@link FlushMode#PERIODIC} mode + */ + int periodicFlushLagBlock(); Review Comment: same as above, should this be a `DurationSpec`? ########## src/java/org/apache/cassandra/journal/Journal.java: ########## @@ -0,0 +1,644 @@ +/* + * 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.journal; + +import java.io.IOException; +import java.nio.file.FileStore; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; +import java.util.zip.CRC32; + +import javax.annotation.Nonnull; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Timer.Context; +import org.apache.cassandra.concurrent.Interruptible.TerminateException; +import org.apache.cassandra.concurrent.Interruptible; +import org.apache.cassandra.concurrent.SequentialExecutorPlus; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.PathUtils; +import org.apache.cassandra.journal.Segments.ReferencedSegments; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.Crc; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.concurrent.WaitQueue; + +import static java.lang.String.format; +import static java.util.Comparator.comparing; +import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.NON_DAEMON; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.SYNCHRONIZED; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; +import static org.apache.cassandra.concurrent.Interruptible.State.NORMAL; +import static org.apache.cassandra.concurrent.Interruptible.State.SHUTTING_DOWN; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue; + +/** + * A generic append-only journal with some special features: + * <p><ul> + * <li>Records can be looked up by key + * <li>Records can be tagged with multiple owner node ids + * <li>Records can be invalidated by their owner ids + * <li>Fully invalidated records get purged during segment compaction + * </ul><p> + * + * Type parameters: + * @param <V> the type of records stored in the journal + * @param <K> the type of keys used to address the records; + must be fixed-size and byte-order comparable + */ +public class Journal<K, V> +{ + private static final Logger logger = LoggerFactory.getLogger(Journal.class); + + final String name; + final File directory; + final Params params; + + final KeySupport<K> keySupport; + final ValueSerializer<K, V> valueSerializer; + + final Metrics<K, V> metrics; + final Flusher<K, V> flusher; + //final Invalidator<K, V> invalidator; + //final Compactor<K, V> compactor; + + volatile long replayLimit; + final AtomicLong nextSegmentId = new AtomicLong(); + + private volatile ActiveSegment<K> currentSegment = null; + + // segment that is ready to be used; allocator thread fills this and blocks until consumed + private volatile ActiveSegment<K> availableSegment = null; + + private final AtomicReference<Segments<K>> segments = new AtomicReference<>(); + + Interruptible allocator; + private final WaitQueue segmentPrepared = newWaitQueue(); + private final WaitQueue allocatorThreadWaitQueue = newWaitQueue(); + private final BooleanSupplier allocatorThreadWaitCondition = () -> (availableSegment == null); + + SequentialExecutorPlus closer; + //private final Set<Descriptor> invalidations = Collections.newSetFromMap(new ConcurrentHashMap<>()); + + public Journal(String name, + File directory, + Params params, + KeySupport<K> keySupport, + ValueSerializer<K, V> valueSerializer) + { + this.name = name; + this.directory = directory; + this.params = params; + + this.keySupport = keySupport; + this.valueSerializer = valueSerializer; + + this.metrics = new Metrics<>(name); + this.flusher = new Flusher<>(this); + //this.invalidator = new Invalidator<>(this); + //this.compactor = new Compactor<>(this); + } + + public void start() + { + metrics.register(flusher); + + deleteTmpFiles(); + + List<Descriptor> descriptors = Descriptor.list(directory); + // find the largest existing timestamp + descriptors.sort(null); + long maxTimestamp = descriptors.isEmpty() + ? Long.MIN_VALUE + : descriptors.get(descriptors.size() - 1).timestamp; + nextSegmentId.set(replayLimit = Math.max(currentTimeMillis(), maxTimestamp + 1)); + + segments.set(Segments.ofStatic(StaticSegment.open(descriptors, keySupport))); + closer = executorFactory().sequential(name + "-closer"); + allocator = executorFactory().infiniteLoop(name + "-allocator", new AllocateRunnable(), SAFE, NON_DAEMON, SYNCHRONIZED); + advanceSegment(null); + flusher.start(); + //invalidator.start(); + //compactor.start(); + } + + /** + * Cleans up unfinished component files from previous run (metadata and index) + */ + private void deleteTmpFiles() + { + for (File tmpFile : directory.listUnchecked(Descriptor::isTmpFile)) + tmpFile.delete(); + } + + public void shutdown() + { + allocator.shutdown(); + //compactor.stop(); + //invalidator.stop(); + flusher.shutdown(); + closer.shutdown(); + closeAllSegments(); + metrics.deregister(); + } + + /** + * Looks up a record by the provided id. + * <p/> + * Looking up an invalidated record may or may not return a record, depending on + * compaction progress. + * <p/> + * In case multiple copies of the record exist in the log (e.g. because of user retries), + * only the first found record will be consumed. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param consumer function to consume the raw record (bytes and invalidation set) if found + * @return true if the record was found, false otherwise + */ + public boolean read(K id, RecordConsumer<K> consumer) + { + try (ReferencedSegments<K> segments = selectAndReference(id)) + { + for (Segment<K> segment : segments.all()) + if (segment.read(id, consumer)) + return true; + } + + return false; + } + + /** + * Looks up a record by the provided id. + * <p/> + * Looking up an invalidated record may or may not return a record, depending on + * compaction progress. + * <p/> + * In case multiple copies of the record exist in the log (e.g. because of user retries), + * the first one found will be returned. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @return deserialized record if found, null otherwise + */ + public V read(K id) + { + EntrySerializer.EntryHolder<K> holder = new EntrySerializer.EntryHolder<>(); + + try (ReferencedSegments<K> segments = selectAndReference(id)) + { + for (Segment<K> segment : segments.all()) + { + if (segment.read(id, holder)) + { + try + { + return valueSerializer.deserialize(holder.key, new DataInputBuffer(holder.value, false), segment.descriptor.userVersion); + } + catch (IOException e) + { + // can only throw if serializer is buggy + throw new RuntimeException(e); + } + } + } + } + + return null; + } + + /** + * Synchronously write a record to the journal. + * <p/> + * Blocks until the record has been deemed durable according to the journal flush mode. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param record the record to store + * @param hosts hosts expected to invalidate the record + */ + public void write(K id, V record, Set<Integer> hosts) + { + try (DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get()) + { + valueSerializer.serialize(record, dob, MessagingService.current_version); + ActiveSegment<K>.Allocation alloc = allocate(dob.getLength(), hosts); + alloc.write(id, dob.unsafeGetBufferAndFlip(), hosts); + flusher.waitForFlush(alloc); + } + catch (IOException e) + { + // exception during record serialization into the scratch buffer + throw new RuntimeException(e); + } + } + + /** + * Asynchronously write a record to the journal. Writes to the journal in the calling thread, + * but doesn't wait for flush. + * <p/> + * Executes the supplied callback on the executor provided, + * once the record has been deemed durable according to the journal flush mode. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param record the record to store + * @param hosts hosts expected to invalidate the record + */ + public void asyncWrite(K id, V record, Set<Integer> hosts, @Nonnull Executor executor, @Nonnull Runnable onDurable) + { + try (DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get()) + { + valueSerializer.serialize(record, dob, MessagingService.current_version); Review Comment: ```suggestion valueSerializer.serialize(record, dob, params.userVersion()); ``` ########## src/java/org/apache/cassandra/journal/Params.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.journal; + +public interface Params +{ + enum FlushMode { BATCH, GROUP, PERIODIC } + + enum FailurePolicy { STOP, STOP_JOURNAL, IGNORE, DIE } + + /** + * @return maximum segment size + */ + int segmentSize(); + + /** + * @return this journal's {@link FailurePolicy} + */ + FailurePolicy failurePolicy(); + + /** + * @return journal flush (sync) mode + */ + FlushMode flushMode(); + + /** + * @return milliseconds between journal flushes + */ + int flushPeriod(); Review Comment: rather than use `int` should we use a `Duration` (actually `DurationSpec`)? Thinking about the yaml side where we would want `flush_period: 1s` ########## src/java/org/apache/cassandra/journal/Descriptor.java: ########## @@ -0,0 +1,167 @@ +/* + * 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.journal; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.cassandra.io.util.File; + +import static java.lang.String.format; +import static java.util.stream.Collectors.toList; + +/** + * Timestamp and version encoded in the file name, e.g. + * log-1637159888484-2-1-1.data + * log-1637159888484-2-1-1.indx + * log-1637159888484-2-1-1.meta + * log-1637159888484-2-1-1.sync + */ +final class Descriptor implements Comparable<Descriptor> +{ + private static final String SEPARATOR = "-"; + private static final String PREFIX = "log" + SEPARATOR; + private static final String TMP_SUFFIX = "tmp"; + + private static final Pattern DATA_FILE_PATTERN = + Pattern.compile( PREFIX + "(\\d+)" // timestamp + + SEPARATOR + "(\\d+)" // generation + + SEPARATOR + "(\\d+)" // journal version + + SEPARATOR + "(\\d+)" // user version + + "\\." + Component.DATA.extension); + + private static final Pattern TMP_FILE_PATTERN = + Pattern.compile( PREFIX + "\\d+" // timestamp + + SEPARATOR + "\\d+" // generation + + SEPARATOR + "\\d+" // journal version + + SEPARATOR + "\\d+" // user version + + "\\." + "[a-z]+" // component extension + + "\\." + TMP_SUFFIX); + + + static final int JOURNAL_VERSION_1 = 1; + static final int CURRENT_JOURNAL_VERSION = JOURNAL_VERSION_1; + + final File directory; + final long timestamp; + final int generation; + + /** + * Serialization version for journal components; bumped as journal + * implementation evolves over time. + */ + final int journalVersion; + + /** + * Serialization version for user content - specifically journal keys + * and journal values; bumped when user logic evolves. + */ + final int userVersion; + + private Descriptor(File directory, long timestamp, int generation, int journalVersion, int userVersion) + { + this.directory = directory; + this.timestamp = timestamp; + this.generation = generation; + this.journalVersion = journalVersion; + this.userVersion = userVersion; + } + + static Descriptor create(File directory, long timestamp, int userVersion) + { + return new Descriptor(directory, timestamp, 1, CURRENT_JOURNAL_VERSION, userVersion); + } + + static Descriptor fromName(File directory, String name) + { + Matcher matcher = DATA_FILE_PATTERN.matcher(name); + if (!matcher.matches()) + throw new IllegalArgumentException("Provided filename is not valid for a data segment file"); + + long timestamp = Long.parseLong(matcher.group(1)); + int generation = Integer.parseInt(matcher.group(2)); + int journalVersion = Integer.parseInt(matcher.group(3)); + int userVersion = Integer.parseInt(matcher.group(4)); + + return new Descriptor(directory, timestamp, generation, journalVersion, userVersion); + } + + Descriptor withIncrementedGeneration() + { + return new Descriptor(directory, timestamp, generation + 1, journalVersion, userVersion); + } + + File fileFor(Component component) + { + return new File(directory, formatFileName(component)); + } + + File tmpFileFor(Component component) + { + return new File(directory, formatFileName(component) + '.' + TMP_SUFFIX); + } + + static boolean isTmpFile(File file) + { + return TMP_FILE_PATTERN.matcher(file.name()).matches(); + } + + private String formatFileName(Component component) + { + return format("%s%d%s%d%s%d%s%d.%s", + PREFIX, timestamp, + SEPARATOR, generation, + SEPARATOR, journalVersion, + SEPARATOR, userVersion, + component.extension); + } + + static List<Descriptor> list(File directory) + { + try + { + return Arrays.stream(directory.listNames((file, name) -> DATA_FILE_PATTERN.matcher(name).matches())) + .map(name -> fromName(directory, name)) + .collect(toList()); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + } + + @Override + public int compareTo(Descriptor other) Review Comment: atm we only use a fixed `directory` but we could spread that cross different disks, so would be good to also compare `directory` as well. ########## src/java/org/apache/cassandra/service/accord/AccordService.java: ########## @@ -78,6 +78,7 @@ public class AccordService implements IAccordService, Shutdownable private final AccordConfigurationService configService; private final AccordScheduler scheduler; private final AccordVerbHandler<? extends Request> verbHandler; + private final AccordJournal journal; Review Comment: do we need this reference? think we can drop as this is *owned* by `CommandStores`? ########## src/java/org/apache/cassandra/journal/Journal.java: ########## @@ -0,0 +1,644 @@ +/* + * 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.journal; + +import java.io.IOException; +import java.nio.file.FileStore; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; +import java.util.zip.CRC32; + +import javax.annotation.Nonnull; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Timer.Context; +import org.apache.cassandra.concurrent.Interruptible.TerminateException; +import org.apache.cassandra.concurrent.Interruptible; +import org.apache.cassandra.concurrent.SequentialExecutorPlus; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.PathUtils; +import org.apache.cassandra.journal.Segments.ReferencedSegments; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.Crc; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.concurrent.WaitQueue; + +import static java.lang.String.format; +import static java.util.Comparator.comparing; +import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.NON_DAEMON; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.SYNCHRONIZED; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; +import static org.apache.cassandra.concurrent.Interruptible.State.NORMAL; +import static org.apache.cassandra.concurrent.Interruptible.State.SHUTTING_DOWN; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue; + +/** + * A generic append-only journal with some special features: + * <p><ul> + * <li>Records can be looked up by key + * <li>Records can be tagged with multiple owner node ids + * <li>Records can be invalidated by their owner ids + * <li>Fully invalidated records get purged during segment compaction + * </ul><p> + * + * Type parameters: + * @param <V> the type of records stored in the journal + * @param <K> the type of keys used to address the records; + must be fixed-size and byte-order comparable + */ +public class Journal<K, V> +{ + private static final Logger logger = LoggerFactory.getLogger(Journal.class); + + final String name; + final File directory; + final Params params; + + final KeySupport<K> keySupport; + final ValueSerializer<K, V> valueSerializer; + + final Metrics<K, V> metrics; + final Flusher<K, V> flusher; + //final Invalidator<K, V> invalidator; + //final Compactor<K, V> compactor; + + volatile long replayLimit; + final AtomicLong nextSegmentId = new AtomicLong(); + + private volatile ActiveSegment<K> currentSegment = null; + + // segment that is ready to be used; allocator thread fills this and blocks until consumed + private volatile ActiveSegment<K> availableSegment = null; + + private final AtomicReference<Segments<K>> segments = new AtomicReference<>(); + + Interruptible allocator; + private final WaitQueue segmentPrepared = newWaitQueue(); + private final WaitQueue allocatorThreadWaitQueue = newWaitQueue(); + private final BooleanSupplier allocatorThreadWaitCondition = () -> (availableSegment == null); + + SequentialExecutorPlus closer; + //private final Set<Descriptor> invalidations = Collections.newSetFromMap(new ConcurrentHashMap<>()); + + public Journal(String name, + File directory, + Params params, + KeySupport<K> keySupport, + ValueSerializer<K, V> valueSerializer) + { + this.name = name; + this.directory = directory; + this.params = params; + + this.keySupport = keySupport; + this.valueSerializer = valueSerializer; + + this.metrics = new Metrics<>(name); + this.flusher = new Flusher<>(this); + //this.invalidator = new Invalidator<>(this); + //this.compactor = new Compactor<>(this); + } + + public void start() + { + metrics.register(flusher); + + deleteTmpFiles(); + + List<Descriptor> descriptors = Descriptor.list(directory); + // find the largest existing timestamp + descriptors.sort(null); + long maxTimestamp = descriptors.isEmpty() + ? Long.MIN_VALUE + : descriptors.get(descriptors.size() - 1).timestamp; + nextSegmentId.set(replayLimit = Math.max(currentTimeMillis(), maxTimestamp + 1)); + + segments.set(Segments.ofStatic(StaticSegment.open(descriptors, keySupport))); + closer = executorFactory().sequential(name + "-closer"); + allocator = executorFactory().infiniteLoop(name + "-allocator", new AllocateRunnable(), SAFE, NON_DAEMON, SYNCHRONIZED); + advanceSegment(null); + flusher.start(); + //invalidator.start(); + //compactor.start(); + } + + /** + * Cleans up unfinished component files from previous run (metadata and index) + */ + private void deleteTmpFiles() + { + for (File tmpFile : directory.listUnchecked(Descriptor::isTmpFile)) + tmpFile.delete(); + } + + public void shutdown() + { + allocator.shutdown(); + //compactor.stop(); + //invalidator.stop(); + flusher.shutdown(); + closer.shutdown(); + closeAllSegments(); + metrics.deregister(); + } + + /** + * Looks up a record by the provided id. + * <p/> + * Looking up an invalidated record may or may not return a record, depending on + * compaction progress. + * <p/> + * In case multiple copies of the record exist in the log (e.g. because of user retries), + * only the first found record will be consumed. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param consumer function to consume the raw record (bytes and invalidation set) if found + * @return true if the record was found, false otherwise + */ + public boolean read(K id, RecordConsumer<K> consumer) + { + try (ReferencedSegments<K> segments = selectAndReference(id)) + { + for (Segment<K> segment : segments.all()) + if (segment.read(id, consumer)) + return true; + } + + return false; + } + + /** + * Looks up a record by the provided id. + * <p/> + * Looking up an invalidated record may or may not return a record, depending on + * compaction progress. + * <p/> + * In case multiple copies of the record exist in the log (e.g. because of user retries), + * the first one found will be returned. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @return deserialized record if found, null otherwise + */ + public V read(K id) + { + EntrySerializer.EntryHolder<K> holder = new EntrySerializer.EntryHolder<>(); + + try (ReferencedSegments<K> segments = selectAndReference(id)) + { + for (Segment<K> segment : segments.all()) + { + if (segment.read(id, holder)) + { + try + { + return valueSerializer.deserialize(holder.key, new DataInputBuffer(holder.value, false), segment.descriptor.userVersion); + } + catch (IOException e) + { + // can only throw if serializer is buggy + throw new RuntimeException(e); + } + } + } + } + + return null; + } + + /** + * Synchronously write a record to the journal. + * <p/> + * Blocks until the record has been deemed durable according to the journal flush mode. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param record the record to store + * @param hosts hosts expected to invalidate the record + */ + public void write(K id, V record, Set<Integer> hosts) + { + try (DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get()) + { + valueSerializer.serialize(record, dob, MessagingService.current_version); + ActiveSegment<K>.Allocation alloc = allocate(dob.getLength(), hosts); + alloc.write(id, dob.unsafeGetBufferAndFlip(), hosts); + flusher.waitForFlush(alloc); + } + catch (IOException e) + { + // exception during record serialization into the scratch buffer + throw new RuntimeException(e); + } + } + + /** + * Asynchronously write a record to the journal. Writes to the journal in the calling thread, + * but doesn't wait for flush. + * <p/> + * Executes the supplied callback on the executor provided, + * once the record has been deemed durable according to the journal flush mode. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param record the record to store + * @param hosts hosts expected to invalidate the record + */ + public void asyncWrite(K id, V record, Set<Integer> hosts, @Nonnull Executor executor, @Nonnull Runnable onDurable) Review Comment: I also know this doesn't exist outside of Accord, but `AsyncChain` helps with this pattern... so could have the following signature ``` public AsyncChain<Void> asyncWrite(K id, V record, Set<Integer> hosts) ``` This allows the caller to control things with regard to the callback better, such as what executor is used (`addCallback(callback, executor)`) or what to do after the option complete. This does not solve the concern that errors do not propagate to callbacks, that is still an issue, this just changes how callbacks are used ########## src/java/org/apache/cassandra/journal/Params.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.journal; + +public interface Params Review Comment: is this meant to be used in `cassandra.yaml`? If so why not just make this a class with the defaults from `DEFAULT`? https://issues.apache.org/jira/browse/CASSANDRA-18221 is in the works now, that patch will add configs to Accord and plumb that into C*. One feature that will exist there is a new annotation of `@Embedded` which will let you have complex/nested types but they get "embedded" into the config yaml... so in your case I think you want ``` // in Config.java @Embedded public final Params accord_journal = new Params; ``` and the yaml would be ``` accord_journal_segment_size: 33554432 accord_journal_failure_policy: STOP ... ``` ########## src/java/org/apache/cassandra/service/accord/async/AsyncAppender.java: ########## @@ -0,0 +1,82 @@ +/* + * 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.service.accord.async; + +import java.util.function.BiConsumer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.local.PreLoadContext; +import org.apache.cassandra.service.accord.AccordCommandStore; + +/** + * Durably appends PreAccept, Accept, Commit, and Apply messages to {@code AccordJournal} + * before any further steps can be attempted. + */ +public class AsyncAppender implements Runnable +{ + private static final Logger logger = LoggerFactory.getLogger(AsyncAppender.class); + + enum State { INITIALIZED, WAITING, FINISHED } + private State state = State.INITIALIZED; + + private final AccordCommandStore commandStore; + private final PreLoadContext preLoadContext; + private final BiConsumer<Object, Throwable> callback; + + public AsyncAppender(AccordCommandStore commandStore, PreLoadContext preLoadContext, BiConsumer<Object, Throwable> callback) + { + this.commandStore = commandStore; + this.preLoadContext = preLoadContext; + this.callback = callback; + } + + public boolean append() + { + if (!commandStore.mustAppendToJournal(preLoadContext)) + { + logger.trace("Skipping append for {}: {}", callback, preLoadContext); + return true; + } + + commandStore.checkInStoreThread(); + logger.trace("Running append for {} with state {}: {}", callback, state, preLoadContext); + switch (state) + { + case INITIALIZED: + commandStore.appendToJournal(preLoadContext, this); + state = State.WAITING; + case WAITING: + case FINISHED: + break; + } + logger.trace("Exiting append for {} with state {}: {}", callback, state, preLoadContext); + return state == State.FINISHED; + } + + @Override + public void run() + { + commandStore.checkInStoreThread(); + if (state != State.WAITING) + throw new IllegalStateException(); Review Comment: ```suggestion Preconditions.checkState(state == State.WAITING, "Expected WAITING state but was %s", state); ``` Or current behavior, just with a useful error message ########## src/java/org/apache/cassandra/journal/Journal.java: ########## @@ -0,0 +1,644 @@ +/* + * 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.journal; + +import java.io.IOException; +import java.nio.file.FileStore; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; +import java.util.zip.CRC32; + +import javax.annotation.Nonnull; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Timer.Context; +import org.apache.cassandra.concurrent.Interruptible.TerminateException; +import org.apache.cassandra.concurrent.Interruptible; +import org.apache.cassandra.concurrent.SequentialExecutorPlus; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.PathUtils; +import org.apache.cassandra.journal.Segments.ReferencedSegments; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.Crc; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.concurrent.WaitQueue; + +import static java.lang.String.format; +import static java.util.Comparator.comparing; +import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.NON_DAEMON; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.SYNCHRONIZED; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; +import static org.apache.cassandra.concurrent.Interruptible.State.NORMAL; +import static org.apache.cassandra.concurrent.Interruptible.State.SHUTTING_DOWN; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue; + +/** + * A generic append-only journal with some special features: + * <p><ul> + * <li>Records can be looked up by key + * <li>Records can be tagged with multiple owner node ids + * <li>Records can be invalidated by their owner ids + * <li>Fully invalidated records get purged during segment compaction + * </ul><p> + * + * Type parameters: + * @param <V> the type of records stored in the journal + * @param <K> the type of keys used to address the records; + must be fixed-size and byte-order comparable + */ +public class Journal<K, V> +{ + private static final Logger logger = LoggerFactory.getLogger(Journal.class); + + final String name; + final File directory; + final Params params; + + final KeySupport<K> keySupport; + final ValueSerializer<K, V> valueSerializer; + + final Metrics<K, V> metrics; + final Flusher<K, V> flusher; + //final Invalidator<K, V> invalidator; + //final Compactor<K, V> compactor; + + volatile long replayLimit; + final AtomicLong nextSegmentId = new AtomicLong(); + + private volatile ActiveSegment<K> currentSegment = null; + + // segment that is ready to be used; allocator thread fills this and blocks until consumed + private volatile ActiveSegment<K> availableSegment = null; + + private final AtomicReference<Segments<K>> segments = new AtomicReference<>(); + + Interruptible allocator; + private final WaitQueue segmentPrepared = newWaitQueue(); + private final WaitQueue allocatorThreadWaitQueue = newWaitQueue(); + private final BooleanSupplier allocatorThreadWaitCondition = () -> (availableSegment == null); + + SequentialExecutorPlus closer; + //private final Set<Descriptor> invalidations = Collections.newSetFromMap(new ConcurrentHashMap<>()); + + public Journal(String name, + File directory, + Params params, + KeySupport<K> keySupport, + ValueSerializer<K, V> valueSerializer) + { + this.name = name; + this.directory = directory; + this.params = params; + + this.keySupport = keySupport; + this.valueSerializer = valueSerializer; + + this.metrics = new Metrics<>(name); + this.flusher = new Flusher<>(this); + //this.invalidator = new Invalidator<>(this); + //this.compactor = new Compactor<>(this); + } + + public void start() + { + metrics.register(flusher); + + deleteTmpFiles(); + + List<Descriptor> descriptors = Descriptor.list(directory); + // find the largest existing timestamp + descriptors.sort(null); + long maxTimestamp = descriptors.isEmpty() + ? Long.MIN_VALUE + : descriptors.get(descriptors.size() - 1).timestamp; + nextSegmentId.set(replayLimit = Math.max(currentTimeMillis(), maxTimestamp + 1)); + + segments.set(Segments.ofStatic(StaticSegment.open(descriptors, keySupport))); + closer = executorFactory().sequential(name + "-closer"); + allocator = executorFactory().infiniteLoop(name + "-allocator", new AllocateRunnable(), SAFE, NON_DAEMON, SYNCHRONIZED); + advanceSegment(null); + flusher.start(); + //invalidator.start(); + //compactor.start(); + } + + /** + * Cleans up unfinished component files from previous run (metadata and index) + */ + private void deleteTmpFiles() + { + for (File tmpFile : directory.listUnchecked(Descriptor::isTmpFile)) + tmpFile.delete(); + } + + public void shutdown() + { + allocator.shutdown(); + //compactor.stop(); + //invalidator.stop(); + flusher.shutdown(); + closer.shutdown(); + closeAllSegments(); + metrics.deregister(); + } + + /** + * Looks up a record by the provided id. + * <p/> + * Looking up an invalidated record may or may not return a record, depending on + * compaction progress. + * <p/> + * In case multiple copies of the record exist in the log (e.g. because of user retries), + * only the first found record will be consumed. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param consumer function to consume the raw record (bytes and invalidation set) if found + * @return true if the record was found, false otherwise + */ + public boolean read(K id, RecordConsumer<K> consumer) + { + try (ReferencedSegments<K> segments = selectAndReference(id)) + { + for (Segment<K> segment : segments.all()) + if (segment.read(id, consumer)) + return true; + } + + return false; + } + + /** + * Looks up a record by the provided id. + * <p/> + * Looking up an invalidated record may or may not return a record, depending on + * compaction progress. + * <p/> + * In case multiple copies of the record exist in the log (e.g. because of user retries), + * the first one found will be returned. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @return deserialized record if found, null otherwise + */ + public V read(K id) + { + EntrySerializer.EntryHolder<K> holder = new EntrySerializer.EntryHolder<>(); + + try (ReferencedSegments<K> segments = selectAndReference(id)) + { + for (Segment<K> segment : segments.all()) + { + if (segment.read(id, holder)) + { + try + { + return valueSerializer.deserialize(holder.key, new DataInputBuffer(holder.value, false), segment.descriptor.userVersion); + } + catch (IOException e) + { + // can only throw if serializer is buggy + throw new RuntimeException(e); + } + } + } + } + + return null; + } + + /** + * Synchronously write a record to the journal. + * <p/> + * Blocks until the record has been deemed durable according to the journal flush mode. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param record the record to store + * @param hosts hosts expected to invalidate the record + */ + public void write(K id, V record, Set<Integer> hosts) + { + try (DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get()) + { + valueSerializer.serialize(record, dob, MessagingService.current_version); Review Comment: ```suggestion valueSerializer.serialize(record, dob, params.userVersion()); ``` ########## src/java/org/apache/cassandra/journal/Params.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.journal; + +public interface Params +{ + enum FlushMode { BATCH, GROUP, PERIODIC } + + enum FailurePolicy { STOP, STOP_JOURNAL, IGNORE, DIE } + + /** + * @return maximum segment size + */ + int segmentSize(); + + /** + * @return this journal's {@link FailurePolicy} + */ + FailurePolicy failurePolicy(); + + /** + * @return journal flush (sync) mode + */ + FlushMode flushMode(); + + /** + * @return milliseconds between journal flushes + */ + int flushPeriod(); + + /** + * @return milliseconds to block writes for while waiting for a slow disk flush to complete + * when in {@link FlushMode#PERIODIC} mode + */ + int periodicFlushLagBlock(); + + /** + * @return user provided version to use for key and value serialization + */ + int userVersion(); Review Comment: If this should always be `MessagingService`'s version for all users, then we can use `org.apache.cassandra.net.MessagingService.Version`... I added this to get access to versions w/o loading `MessagingService` for tests, but could use in this context as well. ########## src/java/org/apache/cassandra/journal/Journal.java: ########## @@ -0,0 +1,644 @@ +/* + * 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.journal; + +import java.io.IOException; +import java.nio.file.FileStore; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; +import java.util.zip.CRC32; + +import javax.annotation.Nonnull; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Timer.Context; +import org.apache.cassandra.concurrent.Interruptible.TerminateException; +import org.apache.cassandra.concurrent.Interruptible; +import org.apache.cassandra.concurrent.SequentialExecutorPlus; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.PathUtils; +import org.apache.cassandra.journal.Segments.ReferencedSegments; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.Crc; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.concurrent.WaitQueue; + +import static java.lang.String.format; +import static java.util.Comparator.comparing; +import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.NON_DAEMON; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.SYNCHRONIZED; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; +import static org.apache.cassandra.concurrent.Interruptible.State.NORMAL; +import static org.apache.cassandra.concurrent.Interruptible.State.SHUTTING_DOWN; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.utils.concurrent.WaitQueue.newWaitQueue; + +/** + * A generic append-only journal with some special features: + * <p><ul> + * <li>Records can be looked up by key + * <li>Records can be tagged with multiple owner node ids + * <li>Records can be invalidated by their owner ids + * <li>Fully invalidated records get purged during segment compaction + * </ul><p> + * + * Type parameters: + * @param <V> the type of records stored in the journal + * @param <K> the type of keys used to address the records; + must be fixed-size and byte-order comparable + */ +public class Journal<K, V> +{ + private static final Logger logger = LoggerFactory.getLogger(Journal.class); + + final String name; + final File directory; + final Params params; + + final KeySupport<K> keySupport; + final ValueSerializer<K, V> valueSerializer; + + final Metrics<K, V> metrics; + final Flusher<K, V> flusher; + //final Invalidator<K, V> invalidator; + //final Compactor<K, V> compactor; + + volatile long replayLimit; + final AtomicLong nextSegmentId = new AtomicLong(); + + private volatile ActiveSegment<K> currentSegment = null; + + // segment that is ready to be used; allocator thread fills this and blocks until consumed + private volatile ActiveSegment<K> availableSegment = null; + + private final AtomicReference<Segments<K>> segments = new AtomicReference<>(); + + Interruptible allocator; + private final WaitQueue segmentPrepared = newWaitQueue(); + private final WaitQueue allocatorThreadWaitQueue = newWaitQueue(); + private final BooleanSupplier allocatorThreadWaitCondition = () -> (availableSegment == null); + + SequentialExecutorPlus closer; + //private final Set<Descriptor> invalidations = Collections.newSetFromMap(new ConcurrentHashMap<>()); + + public Journal(String name, + File directory, + Params params, + KeySupport<K> keySupport, + ValueSerializer<K, V> valueSerializer) + { + this.name = name; + this.directory = directory; + this.params = params; + + this.keySupport = keySupport; + this.valueSerializer = valueSerializer; + + this.metrics = new Metrics<>(name); + this.flusher = new Flusher<>(this); + //this.invalidator = new Invalidator<>(this); + //this.compactor = new Compactor<>(this); + } + + public void start() + { + metrics.register(flusher); + + deleteTmpFiles(); + + List<Descriptor> descriptors = Descriptor.list(directory); + // find the largest existing timestamp + descriptors.sort(null); + long maxTimestamp = descriptors.isEmpty() + ? Long.MIN_VALUE + : descriptors.get(descriptors.size() - 1).timestamp; + nextSegmentId.set(replayLimit = Math.max(currentTimeMillis(), maxTimestamp + 1)); + + segments.set(Segments.ofStatic(StaticSegment.open(descriptors, keySupport))); + closer = executorFactory().sequential(name + "-closer"); + allocator = executorFactory().infiniteLoop(name + "-allocator", new AllocateRunnable(), SAFE, NON_DAEMON, SYNCHRONIZED); + advanceSegment(null); + flusher.start(); + //invalidator.start(); + //compactor.start(); + } + + /** + * Cleans up unfinished component files from previous run (metadata and index) + */ + private void deleteTmpFiles() + { + for (File tmpFile : directory.listUnchecked(Descriptor::isTmpFile)) + tmpFile.delete(); + } + + public void shutdown() + { + allocator.shutdown(); + //compactor.stop(); + //invalidator.stop(); + flusher.shutdown(); + closer.shutdown(); + closeAllSegments(); + metrics.deregister(); + } + + /** + * Looks up a record by the provided id. + * <p/> + * Looking up an invalidated record may or may not return a record, depending on + * compaction progress. + * <p/> + * In case multiple copies of the record exist in the log (e.g. because of user retries), + * only the first found record will be consumed. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param consumer function to consume the raw record (bytes and invalidation set) if found + * @return true if the record was found, false otherwise + */ + public boolean read(K id, RecordConsumer<K> consumer) + { + try (ReferencedSegments<K> segments = selectAndReference(id)) + { + for (Segment<K> segment : segments.all()) + if (segment.read(id, consumer)) + return true; + } + + return false; + } + + /** + * Looks up a record by the provided id. + * <p/> + * Looking up an invalidated record may or may not return a record, depending on + * compaction progress. + * <p/> + * In case multiple copies of the record exist in the log (e.g. because of user retries), + * the first one found will be returned. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @return deserialized record if found, null otherwise + */ + public V read(K id) + { + EntrySerializer.EntryHolder<K> holder = new EntrySerializer.EntryHolder<>(); + + try (ReferencedSegments<K> segments = selectAndReference(id)) + { + for (Segment<K> segment : segments.all()) + { + if (segment.read(id, holder)) + { + try + { + return valueSerializer.deserialize(holder.key, new DataInputBuffer(holder.value, false), segment.descriptor.userVersion); + } + catch (IOException e) + { + // can only throw if serializer is buggy + throw new RuntimeException(e); + } + } + } + } + + return null; + } + + /** + * Synchronously write a record to the journal. + * <p/> + * Blocks until the record has been deemed durable according to the journal flush mode. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param record the record to store + * @param hosts hosts expected to invalidate the record + */ + public void write(K id, V record, Set<Integer> hosts) + { + try (DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get()) + { + valueSerializer.serialize(record, dob, MessagingService.current_version); + ActiveSegment<K>.Allocation alloc = allocate(dob.getLength(), hosts); + alloc.write(id, dob.unsafeGetBufferAndFlip(), hosts); + flusher.waitForFlush(alloc); + } + catch (IOException e) + { + // exception during record serialization into the scratch buffer + throw new RuntimeException(e); + } + } + + /** + * Asynchronously write a record to the journal. Writes to the journal in the calling thread, + * but doesn't wait for flush. + * <p/> + * Executes the supplied callback on the executor provided, + * once the record has been deemed durable according to the journal flush mode. + * + * @param id user-provided record id, expected to roughly correlate with time and go up + * @param record the record to store + * @param hosts hosts expected to invalidate the record + */ + public void asyncWrite(K id, V record, Set<Integer> hosts, @Nonnull Executor executor, @Nonnull Runnable onDurable) Review Comment: When I see `Runnable` for callback, I tend to ask what happens in the failure case... if I am reading this properly `org.apache.cassandra.journal.Flusher.FlushRunnable#run` sees the failure (so does `org.apache.cassandra.journal.Journal#asyncWrite`), but has no way to notify the blocked logic that the segment couldn't save; this can be a problem as the "chain" of operations going on in Accord never knows that the operation failed and won't ever succeed, so can't properly handle the error case on their side. We should notify if the section of the segment we are waiting on failed for w/e reason and that durability maybe lost ########## src/java/org/apache/cassandra/service/accord/async/AsyncAppender.java: ########## @@ -0,0 +1,82 @@ +/* + * 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.service.accord.async; + +import java.util.function.BiConsumer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.local.PreLoadContext; +import org.apache.cassandra.service.accord.AccordCommandStore; + +/** + * Durably appends PreAccept, Accept, Commit, and Apply messages to {@code AccordJournal} + * before any further steps can be attempted. + */ +public class AsyncAppender implements Runnable +{ + private static final Logger logger = LoggerFactory.getLogger(AsyncAppender.class); + + enum State { INITIALIZED, WAITING, FINISHED } + private State state = State.INITIALIZED; + + private final AccordCommandStore commandStore; + private final PreLoadContext preLoadContext; + private final BiConsumer<Object, Throwable> callback; + + public AsyncAppender(AccordCommandStore commandStore, PreLoadContext preLoadContext, BiConsumer<Object, Throwable> callback) + { + this.commandStore = commandStore; + this.preLoadContext = preLoadContext; + this.callback = callback; + } + + public boolean append() + { + if (!commandStore.mustAppendToJournal(preLoadContext)) + { + logger.trace("Skipping append for {}: {}", callback, preLoadContext); + return true; + } + + commandStore.checkInStoreThread(); + logger.trace("Running append for {} with state {}: {}", callback, state, preLoadContext); + switch (state) + { + case INITIALIZED: + commandStore.appendToJournal(preLoadContext, this); + state = State.WAITING; + case WAITING: + case FINISHED: Review Comment: not needed, can remove -- 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]

