You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2014/07/14 21:12:44 UTC

[1/3] git commit: Remove misguided stack allocation optimisations

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 0396efc41 -> 88f9c92f4
  refs/heads/trunk 461962866 -> 1d508f7f6


Remove misguided stack allocation optimisations

patch by Benedict; reviewed by Aleksey for CASSANDRA-7515


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/88f9c92f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/88f9c92f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/88f9c92f

Branch: refs/heads/cassandra-2.1
Commit: 88f9c92f4561c77b40ca25594f3b926824d506c9
Parents: 0396efc
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Mon Jul 14 20:11:08 2014 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Mon Jul 14 20:11:08 2014 +0100

----------------------------------------------------------------------
 src/java/org/apache/cassandra/db/Keyspace.java  |  8 +---
 src/java/org/apache/cassandra/db/Memtable.java  |  2 -
 .../cassandra/db/commitlog/CommitLog.java       | 12 +-----
 .../db/commitlog/CommitLogSegment.java          | 42 ++++++++++----------
 .../db/commitlog/CommitLogSegmentManager.java   |  5 ++-
 5 files changed, 27 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index ff4d272..4a07f5e 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -364,18 +364,12 @@ public class Keyspace
         try (OpOrder.Group opGroup = writeOrder.start())
         {
             // write the mutation to the commitlog and memtables
-            final ReplayPosition replayPosition;
+            ReplayPosition replayPosition = null;
             if (writeCommitLog)
             {
                 Tracing.trace("Appending to commitlog");
                 replayPosition = CommitLog.instance.add(mutation);
             }
-            else
-            {
-                // we don't need the replayposition, but grab one anyway so that it stays stack allocated.
-                // (the JVM will not stack allocate if the object may be null.)
-                replayPosition = CommitLog.instance.getContext();
-            }
 
             DecoratedKey key = StorageService.getPartitioner().decorateKey(mutation.key());
             for (ColumnFamily cf : mutation.getColumnFamilies())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index 3c02678..1eea915 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -157,8 +157,6 @@ public class Memtable
         {
             // if the writeBarrier is set, we want to maintain lastReplayPosition; this is an optimisation to avoid
             // casing it for every write, but still ensure it is correct when writeBarrier.await() completes.
-            // we clone the replay position so that the object passed in does not "escape", permitting stack allocation
-            replayPosition = replayPosition.clone();
             while (true)
             {
                 ReplayPosition last = lastReplayPosition.get();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index ac1d811..a1be25d 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -21,12 +21,10 @@ import java.io.*;
 import java.lang.management.ManagementFactory;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.TimeUnit;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
-import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -211,12 +209,6 @@ public class CommitLog implements CommitLogMBean
      */
     public ReplayPosition add(Mutation mutation)
     {
-        Allocation alloc = add(mutation, new Allocation());
-        return alloc.getReplayPosition();
-    }
-
-    private Allocation add(Mutation mutation, Allocation alloc)
-    {
         assert mutation != null;
 
         long size = Mutation.serializer.serializedSize(mutation, MessagingService.current_version);
@@ -228,7 +220,7 @@ public class CommitLog implements CommitLogMBean
                                                              totalSize, MAX_MUTATION_SIZE));
         }
 
-        allocator.allocate(mutation, (int) totalSize, alloc);
+        Allocation alloc = allocator.allocate(mutation, (int) totalSize);
         try
         {
             PureJavaCrc32 checksum = new PureJavaCrc32();
@@ -256,7 +248,7 @@ public class CommitLog implements CommitLogMBean
         }
 
         executor.finishWriteFor(alloc);
-        return alloc;
+        return alloc.getReplayPosition();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index a983f01..6f756f7 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -43,7 +43,6 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.CLibrary;
@@ -180,31 +179,26 @@ public class CommitLogSegment
     }
 
     /**
-     * allocate space in this buffer for the provided mutation, and populate the provided
-     * Allocation object, returning true on success. False indicates there is not enough room in
-     * this segment, and a new segment is needed
+     * Allocate space in this buffer for the provided mutation, and return the allocated Allocation object.
+     * Returns null if there is not enough space in this segment, and a new segment is needed.
      */
-    boolean allocate(Mutation mutation, int size, Allocation alloc)
+    Allocation allocate(Mutation mutation, int size)
     {
-        final OpOrder.Group commandOrder = appendOrder.start();
+        final OpOrder.Group opGroup = appendOrder.start();
         try
         {
             int position = allocate(size);
             if (position < 0)
             {
-                commandOrder.close();
-                return false;
+                opGroup.close();
+                return null;
             }
-            alloc.buffer = (ByteBuffer) buffer.duplicate().position(position).limit(position + size);
-            alloc.position = position;
-            alloc.segment = this;
-            alloc.appendOp = commandOrder;
             markDirty(mutation, position);
-            return true;
+            return new Allocation(this, opGroup, position, (ByteBuffer) buffer.duplicate().position(position).limit(position + size));
         }
         catch (Throwable t)
         {
-            commandOrder.close();
+            opGroup.close();
             throw t;
         }
     }
@@ -587,10 +581,18 @@ public class CommitLogSegment
     static class Allocation
     {
 
-        private CommitLogSegment segment;
-        private OpOrder.Group appendOp;
-        private int position;
-        private ByteBuffer buffer;
+        private final CommitLogSegment segment;
+        private final OpOrder.Group appendOp;
+        private final int position;
+        private final ByteBuffer buffer;
+
+        Allocation(CommitLogSegment segment, OpOrder.Group appendOp, int position, ByteBuffer buffer)
+        {
+            this.segment = segment;
+            this.appendOp = appendOp;
+            this.position = position;
+            this.buffer = buffer;
+        }
 
         CommitLogSegment getSegment()
         {
@@ -623,9 +625,7 @@ public class CommitLogSegment
 
         public ReplayPosition getReplayPosition()
         {
-            // always allocate a ReplayPosition to let stack allocation do its magic. If we return null, we always
-            // have to allocate an object on the stack
-            return new ReplayPosition(segment == null ? -1 : segment.id, segment == null ? 0 : buffer.limit());
+            return new ReplayPosition(segment.id, buffer.limit());
         }
 
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index e1a7e39..c93d1eb 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -176,11 +176,12 @@ public class CommitLogSegmentManager
      *
      * @return the provided Allocation object
      */
-    public Allocation allocate(Mutation mutation, int size, Allocation alloc)
+    public Allocation allocate(Mutation mutation, int size)
     {
         CommitLogSegment segment = allocatingFrom();
 
-        while (!segment.allocate(mutation, size, alloc))
+        Allocation alloc;
+        while ( null == (alloc = segment.allocate(mutation, size)) )
         {
             // failed to allocate, so move to a new segment with enough room
             advanceAllocatingFrom(segment);


[2/3] git commit: Remove misguided stack allocation optimisations

Posted by be...@apache.org.
Remove misguided stack allocation optimisations

patch by Benedict; reviewed by Aleksey for CASSANDRA-7515


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/88f9c92f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/88f9c92f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/88f9c92f

Branch: refs/heads/trunk
Commit: 88f9c92f4561c77b40ca25594f3b926824d506c9
Parents: 0396efc
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Mon Jul 14 20:11:08 2014 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Mon Jul 14 20:11:08 2014 +0100

----------------------------------------------------------------------
 src/java/org/apache/cassandra/db/Keyspace.java  |  8 +---
 src/java/org/apache/cassandra/db/Memtable.java  |  2 -
 .../cassandra/db/commitlog/CommitLog.java       | 12 +-----
 .../db/commitlog/CommitLogSegment.java          | 42 ++++++++++----------
 .../db/commitlog/CommitLogSegmentManager.java   |  5 ++-
 5 files changed, 27 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index ff4d272..4a07f5e 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -364,18 +364,12 @@ public class Keyspace
         try (OpOrder.Group opGroup = writeOrder.start())
         {
             // write the mutation to the commitlog and memtables
-            final ReplayPosition replayPosition;
+            ReplayPosition replayPosition = null;
             if (writeCommitLog)
             {
                 Tracing.trace("Appending to commitlog");
                 replayPosition = CommitLog.instance.add(mutation);
             }
-            else
-            {
-                // we don't need the replayposition, but grab one anyway so that it stays stack allocated.
-                // (the JVM will not stack allocate if the object may be null.)
-                replayPosition = CommitLog.instance.getContext();
-            }
 
             DecoratedKey key = StorageService.getPartitioner().decorateKey(mutation.key());
             for (ColumnFamily cf : mutation.getColumnFamilies())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index 3c02678..1eea915 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -157,8 +157,6 @@ public class Memtable
         {
             // if the writeBarrier is set, we want to maintain lastReplayPosition; this is an optimisation to avoid
             // casing it for every write, but still ensure it is correct when writeBarrier.await() completes.
-            // we clone the replay position so that the object passed in does not "escape", permitting stack allocation
-            replayPosition = replayPosition.clone();
             while (true)
             {
                 ReplayPosition last = lastReplayPosition.get();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index ac1d811..a1be25d 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -21,12 +21,10 @@ import java.io.*;
 import java.lang.management.ManagementFactory;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.TimeUnit;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
-import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -211,12 +209,6 @@ public class CommitLog implements CommitLogMBean
      */
     public ReplayPosition add(Mutation mutation)
     {
-        Allocation alloc = add(mutation, new Allocation());
-        return alloc.getReplayPosition();
-    }
-
-    private Allocation add(Mutation mutation, Allocation alloc)
-    {
         assert mutation != null;
 
         long size = Mutation.serializer.serializedSize(mutation, MessagingService.current_version);
@@ -228,7 +220,7 @@ public class CommitLog implements CommitLogMBean
                                                              totalSize, MAX_MUTATION_SIZE));
         }
 
-        allocator.allocate(mutation, (int) totalSize, alloc);
+        Allocation alloc = allocator.allocate(mutation, (int) totalSize);
         try
         {
             PureJavaCrc32 checksum = new PureJavaCrc32();
@@ -256,7 +248,7 @@ public class CommitLog implements CommitLogMBean
         }
 
         executor.finishWriteFor(alloc);
-        return alloc;
+        return alloc.getReplayPosition();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index a983f01..6f756f7 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -43,7 +43,6 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.CLibrary;
@@ -180,31 +179,26 @@ public class CommitLogSegment
     }
 
     /**
-     * allocate space in this buffer for the provided mutation, and populate the provided
-     * Allocation object, returning true on success. False indicates there is not enough room in
-     * this segment, and a new segment is needed
+     * Allocate space in this buffer for the provided mutation, and return the allocated Allocation object.
+     * Returns null if there is not enough space in this segment, and a new segment is needed.
      */
-    boolean allocate(Mutation mutation, int size, Allocation alloc)
+    Allocation allocate(Mutation mutation, int size)
     {
-        final OpOrder.Group commandOrder = appendOrder.start();
+        final OpOrder.Group opGroup = appendOrder.start();
         try
         {
             int position = allocate(size);
             if (position < 0)
             {
-                commandOrder.close();
-                return false;
+                opGroup.close();
+                return null;
             }
-            alloc.buffer = (ByteBuffer) buffer.duplicate().position(position).limit(position + size);
-            alloc.position = position;
-            alloc.segment = this;
-            alloc.appendOp = commandOrder;
             markDirty(mutation, position);
-            return true;
+            return new Allocation(this, opGroup, position, (ByteBuffer) buffer.duplicate().position(position).limit(position + size));
         }
         catch (Throwable t)
         {
-            commandOrder.close();
+            opGroup.close();
             throw t;
         }
     }
@@ -587,10 +581,18 @@ public class CommitLogSegment
     static class Allocation
     {
 
-        private CommitLogSegment segment;
-        private OpOrder.Group appendOp;
-        private int position;
-        private ByteBuffer buffer;
+        private final CommitLogSegment segment;
+        private final OpOrder.Group appendOp;
+        private final int position;
+        private final ByteBuffer buffer;
+
+        Allocation(CommitLogSegment segment, OpOrder.Group appendOp, int position, ByteBuffer buffer)
+        {
+            this.segment = segment;
+            this.appendOp = appendOp;
+            this.position = position;
+            this.buffer = buffer;
+        }
 
         CommitLogSegment getSegment()
         {
@@ -623,9 +625,7 @@ public class CommitLogSegment
 
         public ReplayPosition getReplayPosition()
         {
-            // always allocate a ReplayPosition to let stack allocation do its magic. If we return null, we always
-            // have to allocate an object on the stack
-            return new ReplayPosition(segment == null ? -1 : segment.id, segment == null ? 0 : buffer.limit());
+            return new ReplayPosition(segment.id, buffer.limit());
         }
 
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/88f9c92f/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index e1a7e39..c93d1eb 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -176,11 +176,12 @@ public class CommitLogSegmentManager
      *
      * @return the provided Allocation object
      */
-    public Allocation allocate(Mutation mutation, int size, Allocation alloc)
+    public Allocation allocate(Mutation mutation, int size)
     {
         CommitLogSegment segment = allocatingFrom();
 
-        while (!segment.allocate(mutation, size, alloc))
+        Allocation alloc;
+        while ( null == (alloc = segment.allocate(mutation, size)) )
         {
             // failed to allocate, so move to a new segment with enough room
             advanceAllocatingFrom(segment);


[3/3] git commit: Merge branch 'cassandra-2.1' into trunk

Posted by be...@apache.org.
Merge branch 'cassandra-2.1' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1d508f7f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1d508f7f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1d508f7f

Branch: refs/heads/trunk
Commit: 1d508f7f66a31d7a3d0ccf31b977fa6224170bc4
Parents: 4619628 88f9c92
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Mon Jul 14 20:12:18 2014 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Mon Jul 14 20:12:18 2014 +0100

----------------------------------------------------------------------
 src/java/org/apache/cassandra/db/Keyspace.java  |  8 +---
 src/java/org/apache/cassandra/db/Memtable.java  |  2 -
 .../cassandra/db/commitlog/CommitLog.java       | 12 +-----
 .../db/commitlog/CommitLogSegment.java          | 41 ++++++++++----------
 .../db/commitlog/CommitLogSegmentManager.java   |  5 ++-
 5 files changed, 27 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1d508f7f/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1d508f7f/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1d508f7f/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------