dcapwell commented on code in PR #2256:
URL: https://github.com/apache/cassandra/pull/2256#discussion_r1165831519


##########
src/java/org/apache/cassandra/journal/Journal.java:
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.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))

Review Comment:
   this deadlocks if the journal was shutdown
   
   ```
   "begin:1@5677" prio=5 tid=0x2b nid=NA runnable
     java.lang.Thread.State: RUNNABLE
          at 
com.google.common.collect.Iterators.emptyIterator(Iterators.java:77)
          at 
com.google.common.collect.Iterators$ConcatenatedIterator.<init>(Iterators.java:1305)
          at com.google.common.collect.Iterators.concat(Iterators.java:548)
          at 
com.google.common.collect.FluentIterable$3.iterator(FluentIterable.java:291)
          at org.apache.cassandra.utils.concurrent.Refs.tryRef(Refs.java:215)
          at 
org.apache.cassandra.journal.Segments.selectAndReference(Segments.java:168)
          at 
org.apache.cassandra.journal.Journal.selectAndReference(Journal.java:487)
          at org.apache.cassandra.journal.Journal.read(Journal.java:212)
          at 
org.apache.cassandra.service.accord.AccordJournal.read(AccordJournal.java:104)
   ```



-- 
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]

Reply via email to