You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "dcapwell (via GitHub)" <gi...@apache.org> on 2023/04/06 18:25:01 UTC

[GitHub] [cassandra] dcapwell commented on a diff in pull request #2256: CASSANDRA-18344 Store PreAccept, Accept, Commit, and Apply in a durable log before processing by CommandStores

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


##########
src/java/org/apache/cassandra/io/util/TrackedDataOutputPlus.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.utils.vint.VIntCoding;
+
+public class TrackedDataOutputPlus implements DataOutputPlus
+{
+    private final DataOutputPlus out;
+    private int position = 0;
+
+    private TrackedDataOutputPlus(DataOutputPlus out)
+    {
+        this.out = out;
+    }
+
+    public static TrackedDataOutputPlus wrap(DataOutputPlus out)
+    {
+        return new TrackedDataOutputPlus(out);
+    }
+
+    @Override
+    public void write(int b) throws IOException
+    {
+        out.write(b);
+        position += 1;
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException
+    {
+        out.write(b);
+        position += b.length;
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException
+    {
+        out.write(b, off, len);
+        position += len;
+    }
+
+    @Override
+    public void writeBoolean(boolean v) throws IOException
+    {
+        out.writeBoolean(v);
+        position += 1;
+    }
+
+    @Override
+    public void writeByte(int v) throws IOException
+    {
+        out.writeByte(v);
+        position += 1;
+    }
+
+    @Override
+    public void writeShort(int v) throws IOException
+    {
+        out.writeShort(v);
+        position += 2;
+    }
+
+    @Override
+    public void writeChar(int v) throws IOException
+    {
+        out.writeChar(v);
+        position += 2;
+    }
+
+    @Override
+    public void writeInt(int v) throws IOException
+    {
+        out.writeInt(v);
+        position += 4;
+    }
+
+    @Override
+    public void writeLong(long v) throws IOException
+    {
+        out.writeLong(v);
+        position += 8;
+    }
+
+    @Override
+    public void writeFloat(float v) throws IOException
+    {
+        out.writeFloat(v);
+        position += 4;
+    }
+
+    @Override
+    public void writeDouble(double v) throws IOException
+    {
+        out.writeDouble(v);
+        position += 8;
+    }
+
+    @Override
+    public void writeBytes(String s) throws IOException
+    {
+        out.writeBytes(s);
+        position += s.length();
+    }
+
+    @Override
+    public void writeChars(String s) throws IOException
+    {
+        out.writeChars(s);
+        position += s.length() * 2;
+    }
+
+    @Override
+    public void writeUTF(String s) throws IOException
+    {
+        UnbufferedDataOutputStreamPlus.writeUTF(s, out);

Review Comment:
   missing `position`



##########
src/java/org/apache/cassandra/io/util/TrackedDataOutputPlus.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.utils.vint.VIntCoding;
+
+public class TrackedDataOutputPlus implements DataOutputPlus
+{
+    private final DataOutputPlus out;
+    private int position = 0;
+
+    private TrackedDataOutputPlus(DataOutputPlus out)
+    {
+        this.out = out;
+    }
+
+    public static TrackedDataOutputPlus wrap(DataOutputPlus out)
+    {
+        return new TrackedDataOutputPlus(out);
+    }
+
+    @Override
+    public void write(int b) throws IOException
+    {
+        out.write(b);
+        position += 1;
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException
+    {
+        out.write(b);
+        position += b.length;
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException
+    {
+        out.write(b, off, len);
+        position += len;
+    }
+
+    @Override
+    public void writeBoolean(boolean v) throws IOException
+    {
+        out.writeBoolean(v);
+        position += 1;
+    }
+
+    @Override
+    public void writeByte(int v) throws IOException
+    {
+        out.writeByte(v);
+        position += 1;
+    }
+
+    @Override
+    public void writeShort(int v) throws IOException
+    {
+        out.writeShort(v);
+        position += 2;
+    }
+
+    @Override
+    public void writeChar(int v) throws IOException
+    {
+        out.writeChar(v);
+        position += 2;
+    }
+
+    @Override
+    public void writeInt(int v) throws IOException
+    {
+        out.writeInt(v);
+        position += 4;
+    }
+
+    @Override
+    public void writeLong(long v) throws IOException
+    {
+        out.writeLong(v);
+        position += 8;
+    }
+
+    @Override
+    public void writeFloat(float v) throws IOException
+    {
+        out.writeFloat(v);
+        position += 4;
+    }
+
+    @Override
+    public void writeDouble(double v) throws IOException
+    {
+        out.writeDouble(v);
+        position += 8;
+    }
+
+    @Override
+    public void writeBytes(String s) throws IOException
+    {
+        out.writeBytes(s);
+        position += s.length();
+    }
+
+    @Override
+    public void writeChars(String s) throws IOException
+    {
+        out.writeChars(s);
+        position += s.length() * 2;
+    }
+
+    @Override
+    public void writeUTF(String s) throws IOException
+    {
+        UnbufferedDataOutputStreamPlus.writeUTF(s, out);
+    }
+
+    @Override
+    public void write(ByteBuffer buffer) throws IOException
+    {
+        out.write(buffer);
+        position += buffer.remaining();

Review Comment:
   to be safe shouldn't this be first?  just incase `out.write` consumes?



##########
src/java/org/apache/cassandra/io/util/TrackedDataOutputPlus.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.utils.vint.VIntCoding;
+
+public class TrackedDataOutputPlus implements DataOutputPlus
+{
+    private final DataOutputPlus out;
+    private int position = 0;
+
+    private TrackedDataOutputPlus(DataOutputPlus out)
+    {
+        this.out = out;
+    }
+
+    public static TrackedDataOutputPlus wrap(DataOutputPlus out)
+    {
+        return new TrackedDataOutputPlus(out);
+    }
+
+    @Override
+    public void write(int b) throws IOException
+    {
+        out.write(b);
+        position += 1;
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException
+    {
+        out.write(b);
+        position += b.length;
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException
+    {
+        out.write(b, off, len);
+        position += len;
+    }
+
+    @Override
+    public void writeBoolean(boolean v) throws IOException
+    {
+        out.writeBoolean(v);
+        position += 1;
+    }
+
+    @Override
+    public void writeByte(int v) throws IOException
+    {
+        out.writeByte(v);
+        position += 1;
+    }
+
+    @Override
+    public void writeShort(int v) throws IOException
+    {
+        out.writeShort(v);
+        position += 2;
+    }
+
+    @Override
+    public void writeChar(int v) throws IOException
+    {
+        out.writeChar(v);
+        position += 2;
+    }
+
+    @Override
+    public void writeInt(int v) throws IOException
+    {
+        out.writeInt(v);
+        position += 4;
+    }
+
+    @Override
+    public void writeLong(long v) throws IOException
+    {
+        out.writeLong(v);
+        position += 8;
+    }
+
+    @Override
+    public void writeFloat(float v) throws IOException
+    {
+        out.writeFloat(v);
+        position += 4;
+    }
+
+    @Override
+    public void writeDouble(double v) throws IOException
+    {
+        out.writeDouble(v);
+        position += 8;
+    }
+
+    @Override
+    public void writeBytes(String s) throws IOException
+    {
+        out.writeBytes(s);
+        position += s.length();
+    }
+
+    @Override
+    public void writeChars(String s) throws IOException
+    {
+        out.writeChars(s);
+        position += s.length() * 2;
+    }
+
+    @Override
+    public void writeUTF(String s) throws IOException
+    {
+        UnbufferedDataOutputStreamPlus.writeUTF(s, out);
+    }
+
+    @Override
+    public void write(ByteBuffer buffer) throws IOException
+    {
+        out.write(buffer);
+        position += buffer.remaining();
+    }
+
+    @Override
+    public void write(ReadableMemory memory, long offset, long length) throws IOException
+    {
+        out.write(memory, offset, length);
+        position += length;
+    }
+
+    @Override
+    public void writeVInt(long i) throws IOException
+    {
+        VIntCoding.writeVInt(i, this);
+    }
+
+    @Override
+    public void writeUnsignedVInt(long i) throws IOException
+    {
+        VIntCoding.writeUnsignedVInt(i, this);

Review Comment:
   missing `position`



##########
src/java/org/apache/cassandra/io/util/TrackedDataOutputPlus.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.utils.vint.VIntCoding;
+
+public class TrackedDataOutputPlus implements DataOutputPlus
+{
+    private final DataOutputPlus out;
+    private int position = 0;
+
+    private TrackedDataOutputPlus(DataOutputPlus out)
+    {
+        this.out = out;
+    }
+
+    public static TrackedDataOutputPlus wrap(DataOutputPlus out)
+    {
+        return new TrackedDataOutputPlus(out);
+    }
+
+    @Override
+    public void write(int b) throws IOException
+    {
+        out.write(b);
+        position += 1;
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException
+    {
+        out.write(b);
+        position += b.length;
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException
+    {
+        out.write(b, off, len);
+        position += len;
+    }
+
+    @Override
+    public void writeBoolean(boolean v) throws IOException
+    {
+        out.writeBoolean(v);
+        position += 1;
+    }
+
+    @Override
+    public void writeByte(int v) throws IOException
+    {
+        out.writeByte(v);
+        position += 1;
+    }
+
+    @Override
+    public void writeShort(int v) throws IOException
+    {
+        out.writeShort(v);
+        position += 2;
+    }
+
+    @Override
+    public void writeChar(int v) throws IOException
+    {
+        out.writeChar(v);
+        position += 2;
+    }
+
+    @Override
+    public void writeInt(int v) throws IOException
+    {
+        out.writeInt(v);
+        position += 4;
+    }
+
+    @Override
+    public void writeLong(long v) throws IOException
+    {
+        out.writeLong(v);
+        position += 8;
+    }
+
+    @Override
+    public void writeFloat(float v) throws IOException
+    {
+        out.writeFloat(v);
+        position += 4;
+    }
+
+    @Override
+    public void writeDouble(double v) throws IOException
+    {
+        out.writeDouble(v);
+        position += 8;
+    }
+
+    @Override
+    public void writeBytes(String s) throws IOException
+    {
+        out.writeBytes(s);
+        position += s.length();
+    }
+
+    @Override
+    public void writeChars(String s) throws IOException
+    {
+        out.writeChars(s);
+        position += s.length() * 2;
+    }
+
+    @Override
+    public void writeUTF(String s) throws IOException
+    {
+        UnbufferedDataOutputStreamPlus.writeUTF(s, out);
+    }
+
+    @Override
+    public void write(ByteBuffer buffer) throws IOException
+    {
+        out.write(buffer);
+        position += buffer.remaining();
+    }
+
+    @Override
+    public void write(ReadableMemory memory, long offset, long length) throws IOException
+    {
+        out.write(memory, offset, length);
+        position += length;
+    }
+
+    @Override
+    public void writeVInt(long i) throws IOException
+    {
+        VIntCoding.writeVInt(i, this);

Review Comment:
   missing `position`



##########
src/java/org/apache/cassandra/utils/NoSpamLogger.java:
##########
@@ -218,6 +218,11 @@ private NoSpamLogger(Logger wrapped, long minInterval, TimeUnit timeUnit)
         minIntervalNanos = timeUnit.toNanos(minInterval);
     }
 
+    public static NoSpamLogger wrap(Logger wrapped, long minInterval, TimeUnit timeUnit)

Review Comment:
   any reason to use this vs `org.apache.cassandra.utils.NoSpamLogger#getLogger`?



##########
src/java/org/apache/cassandra/utils/JVMStabilityInspector.java:
##########
@@ -92,6 +93,11 @@ public static void inspectCommitLogThrowable(Throwable t)
         inspectThrowable(t, JVMStabilityInspector::inspectCommitLogError);
     }
 
+    public static void inspectJournalThrowable(Throwable t, String journalName, FailurePolicy failurePolicy)

Review Comment:
   to stay consistent shouldn't we drop `FailurePolicy` and internally call `DatabaseDescriptor.getCommitFailurePolicy()`?



##########
test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java:
##########
@@ -331,7 +336,8 @@ public static AccordCommandStore createAccordCommandStore(Node.Id node, LongSupp
                                       new AccordAgent(),
                                       null,
                                       cs -> NOOP_PROGRESS_LOG,
-                                      new SingleEpochRanges(topology.rangesForNode(node)));
+                                      new SingleEpochRanges(topology.rangesForNode(node)),
+                                      NOOP_ACCORD_JOURNAL);

Review Comment:
   kinda feel we might want to always run, but may want to run with an in-memory file system (like what simulator does)



-- 
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-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org