You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@activemq.apache.org by franz1981 <gi...@git.apache.org> on 2018/01/05 18:18:45 UTC

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

GitHub user franz1981 opened a pull request:

    https://github.com/apache/activemq-artemis/pull/1752

    ARTEMIS-1586 Reduce GC pressure due to String allocations on Core protocol

    The commit contains:
    - a general purpose interner implementation
    - StringValue/SimpleString internrs specializations
    - TypedProperties keys/values string interning for SessionSendMessage decoding

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/franz1981/activemq-artemis decoder_interners

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/activemq-artemis/pull/1752.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1752
    
----
commit 2dd894c74f6dfe1e9d0579ced225a44661efcb99
Author: Francesco Nigro <ni...@...>
Date:   2018-01-04T14:22:05Z

    ARTEMIS-1586 Reduce GC pressure due to String allocations on Core protocol
    
    The commit contains:
    - a general purpose interner implementation
    - StringValue/SimpleString internrs specializations
    - TypedProperties keys/values string interning for SessionSendMessage decoding

----


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    it has already embedded into https://github.com/apache/activemq-artemis/pull/1757 so I can close it :+1: 


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @michaelandrepearce `StringValue::StringStringValuePool`  seems not used anymore: do you think I could remove it?


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159997984
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    +
    +   private final T[] entries;
    --- End diff --
    
    The implementation you've shared is doing a rather different thing: so probably I need to explain better what this PR is aiming to do instead..


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160007930
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    This is not how escape analysis work sadly: not on the standard Oracle JVM at least. 
    Maybe Graal does a better job here because if allows partial escaping (first occurrence and never after too) while packing instances on the stack.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159980786
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    one interner per session :O ouch - an interner ideally should be shared within JVM so benefits can be reaped anywhere.


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @michaelandrepearce Don't worry Michael I've found (probably) the error: `SimpleString::equals` was broken, now is fixed!
    I hope the rest of the tests will work!


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160014921
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    Argh! My fault...github diff can't expect 100 line shift I think!


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160009754
  
    --- Diff: artemis-core-client/src/main/java/org/apache/activemq/artemis/core/message/impl/CoreMessage.java ---
    @@ -528,8 +543,17 @@ public void decodeHeadersAndProperties(final ByteBuf buffer) {
        private void decodeHeadersAndProperties(final ByteBuf buffer, boolean lazyProperties) {
           messageIDPosition = buffer.readerIndex();
           messageID = buffer.readLong();
    -
    -      address = SimpleString.readNullableSimpleString(buffer);
    +      int b = buffer.readByte();
    +      if (b != DataConstants.NULL) {
    +         final int length = buffer.readInt();
    +         if (keysInterner != null) {
    +            address = keysInterner.intern(buffer, length);
    --- End diff --
    
    If you are having separate pools of last seen SimpleStrings instead of global interning for typed properties keys and values, you should probably pool address's separate from those. Imagine the pool is size 32 default, very quickly be used up and find typedproperty keys are invalidating address's in that last seen pool


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160004663
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    --- End diff --
    
    please use JMH if possible: if not you have to create a volatile static field where write the array[inx] nad read at the end of the benchmark to avoid the JVM to drop the code or use Escape Analysis to unpack the SimpleString on the stack



---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160011868
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    I still think a global interning solution will be more beneficial especially with the other protocols that don't use SimpleString, but more use String then convert to SimpleString to keep ICoreMessage interface, but this will have same impact (especially in multi protocol broker env like with clients on AMQP and CORE).
    
    
    



---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160007029
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    see my note on the other comment, we can avoid the copy and byte[] object on heap too with keeping that on stack


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    Why re-invent the wheel here, guava interners is well battle tested, and exists already.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160008652
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    Ah spotted it, line 432....almost 100 line jump!


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159993394
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    +
    +   private final T[] entries;
    --- End diff --
    
    A decoder for a connection (hopefully if clients do the right thing and re-use them, should hang around a long time) 


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159984911
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -93,6 +94,7 @@ public void putBooleanProperty(final SimpleString key, final boolean value) {
        }
     
        public void putByteProperty(final SimpleString key, final byte value) {
    +      checkCreateProperties();
    --- End diff --
    
    am i missing something this seems like a duplicate method call.


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    I’m always concerned about over optimization’s on the core protocol.  It’s already really fast. 
    
    
    If we wanted to invest in core the best would be to reuse buffers from messages.  
    
    
    I don’t see much gain here TBH. 


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @franz1981 yeah i think that last merge into your branch, it picked something else up, locally - the classic "it works on my pc" lol . I see you've rebased, ill rebase on yours, and then send PR to your's again


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    We can have two commits that’s fine, one for your bits and then mine.
    
    Yup I thankfully backed up your original and mine, that’s what I rebased from master and replayed onto my branch I have. So you could just take from there.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159978720
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    +
    +   private final T[] entries;
    --- End diff --
    
    this doesn't allow GC to occur ever on any object held by this class, if the object is de-referenced. 


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160010591
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    --- End diff --
    
    Surely this is beneficial for clients also, i would look to put the Object pools/caches, down at PacketDecoder level, and have a number of these settable (address, typed property key, typed properly value) .


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @clebertsuconic tbh, i can see the saving on property keys, and address per session.
    
    Also within the consumer and producer (though for producing the saving would be on String to SimpleString for the keys so one extra variation needed), if you think typically a producer would send typically to one or a few destinations and then send a million of them, like wise on message properties, if you set one you're most likely to set it all the time, and then again typically a consumer would see all the same. 
    
    The only bit I'm dubious on its benefit is property values.
    
    Lastly on the implementation detail this i think needs some work/cleanup, though after finding/understanding through the discussion with @franz1981  its more of a last seen cache/pool so I'm more +1 it, just the name made me to expect something different at first.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160006047
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    Ok, going back to previous note: 
    " but it is by avoiding the byte[] that is reduced by far most of the garbage"
    
    We can do this on stack though with lamda's :) 


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @michaelandrepearce Using the last commits seems that something is broken...even the CoreMessageTest isn't passing...


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159976892
  
    --- Diff: artemis-core-client/src/main/java/org/apache/activemq/artemis/core/message/impl/CoreMessage.java ---
    @@ -528,8 +543,17 @@ public void decodeHeadersAndProperties(final ByteBuf buffer) {
        private void decodeHeadersAndProperties(final ByteBuf buffer, boolean lazyProperties) {
           messageIDPosition = buffer.readerIndex();
           messageID = buffer.readLong();
    -
    -      address = SimpleString.readNullableSimpleString(buffer);
    +      int b = buffer.readByte();
    --- End diff --
    
    Shouldnt this logic remain in SimpleString, as you could read strings (especially address in many places)


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159997687
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    I'm not avoiding only that cost, but the decoding and copy too: that's why I'm not using byte[] but directly mapping ByteBuf to intern the keys/values.


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @franz1981 looks good, i also spent some time going through it offline, i sent a PR to your branch with some bits. 


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159986257
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    It has a different usage than a JVM interner.
    It Is more a "per-connection last recently use fixed size pool": I've chosen to call it interner just for brevity eheh


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160006951
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    It is fragile due to how escape analysis work: looking just at the code the array escapes partially so it is not always true that will live on the stack...
    And considering that the intern could steal it...probably it wont's work...
    I've fought many years against these weird things of the JVM dear Michael :(


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159985473
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    It is back-compatible 


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160001898
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    according to diff its showing red, which means its removed.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160010613
  
    --- Diff: artemis-core-client/src/main/java/org/apache/activemq/artemis/core/message/impl/CoreMessage.java ---
    @@ -528,8 +543,17 @@ public void decodeHeadersAndProperties(final ByteBuf buffer) {
        private void decodeHeadersAndProperties(final ByteBuf buffer, boolean lazyProperties) {
           messageIDPosition = buffer.readerIndex();
           messageID = buffer.readLong();
    -
    -      address = SimpleString.readNullableSimpleString(buffer);
    +      int b = buffer.readByte();
    +      if (b != DataConstants.NULL) {
    +         final int length = buffer.readInt();
    +         if (keysInterner != null) {
    +            address = keysInterner.intern(buffer, length);
    --- End diff --
    
    This is a very good point..yes, agree!


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160007505
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    intern only steals it on first occurance so only that one escapes the stack as thats the one held in the intern, all others that are the same as the interned so wouldn't escape it.
    



---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @franz1981 yeah i think there was a conflict in merge that auto resolved that out. I have just done a clean rebase and re-applied your commits and mine onto https://github.com/apache/activemq-artemis/pull/1757 which keeps the committer history.


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    Bit uncertain on this, as you aren't holding weak references, as such never allows GC to clean objects that no longer exist.


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @franz1981 yeah it shouldn't be there.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159985727
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    --- End diff --
    
    Because this is pretty different implementation-wise: it acts similarly to a bloom filter and it has fixed memory foot-print.
    And obviously the major different is in the performances :)


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159985327
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    +
    +   private final T[] entries;
    --- End diff --
    
    When the decoder will be deallocated the (few) instances will be removed or am I missing something?


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160005561
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    sadly we can't do it with ease for different reasons:
    - most byte[] operations are optimized by the JVM to avoid most of the bounds checks, on ByteBuf I've just provided a PR on Netty to improve this thing
    - I wouldn't change SimpleString that is used everywhere (literally) on Artemis
    - the lifecycle of the original ByteBuf is not predicatable: sometimes are pooled sometimes not, offheap/onheap...not that simple honestly


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160002569
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    you're not making a saving of avoiding iterating though the bytes, to check if equal/hash you're still having to process through that bytebuf bringing the bytes (if off heap) onto heap.
    
    All creating SimpleString with data does is simple sets the byte[] as data, no decoding.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159997343
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    +
    +   private final T[] entries;
    --- End diff --
    
    It will have anyway a fixed (and very low) footprint, hence I don't see any problem on it


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @michaelandrepearce @clebertsuconic 
    As promised I've provided a benchmark that can be run with ease directly from the IDE:
    https://github.com/franz1981/activemq-artemis/tree/jmh_interner_benchmarks
    
    The benchmark is this one:
    https://github.com/franz1981/activemq-artemis/blob/3e0b4b8152bed30ba747704a653d0c034ebe19d5/tests/performance-tests/src/test/java/org/apache/activemq/artemis/tests/performance/jmh/pool/SimpleStringInternerBenchmark.java
    
    Some of results of my box:
    ```
    
    Benchmark                                                                              Mode  Cnt         Score         Error   Units
    SimpleStringInternerBenchmark.artemisIntern                                           thrpt   10  15509306.132 ±  568180.609   ops/s
    SimpleStringInternerBenchmark.artemisIntern:·gc.alloc.rate                            thrpt   10        ≈ 10⁻⁴                MB/sec
    SimpleStringInternerBenchmark.artemisIntern:·gc.alloc.rate.norm                       thrpt   10        ≈ 10⁻⁵                  B/op
    SimpleStringInternerBenchmark.artemisIntern:·gc.count                                 thrpt   10           ≈ 0                counts
    SimpleStringInternerBenchmark.artemisIntern3Threads                                   thrpt   10  44734165.507 ± 1868110.790   ops/s
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.alloc.rate                    thrpt   10         0.006 ±       0.016  MB/sec
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.alloc.rate.norm               thrpt   10        ≈ 10⁻⁴                  B/op
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.count                         thrpt   10           ≈ 0                counts
    SimpleStringInternerBenchmark.guavaInterner                                           thrpt   10   6231479.494 ±  313670.700   ops/s
    SimpleStringInternerBenchmark.guavaInterner:·gc.alloc.rate                            thrpt   10       443.572 ±      22.292  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.alloc.rate.norm                       thrpt   10       112.000 ±       0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.PS_Eden_Space                   thrpt   10       445.183 ±      80.501  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.PS_Eden_Space.norm              thrpt   10       112.375 ±      18.859    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.PS_Survivor_Space               thrpt   10         0.073 ±       0.076  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.PS_Survivor_Space.norm          thrpt   10         0.019 ±       0.020    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.count                                 thrpt   10        44.000                counts
    SimpleStringInternerBenchmark.guavaInterner:·gc.time                                  thrpt   10        56.000                    ms
    SimpleStringInternerBenchmark.guavaInterner3Threads                                   thrpt   10  18200947.459 ±  933389.842   ops/s
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.alloc.rate                    thrpt   10      1295.337 ±      66.617  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.alloc.rate.norm               thrpt   10       112.000 ±       0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.PS_Eden_Space           thrpt   10      1323.335 ±     234.954  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.PS_Eden_Space.norm      thrpt   10       114.500 ±      20.365    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.PS_Survivor_Space       thrpt   10         0.081 ±       0.041  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.PS_Survivor_Space.norm  thrpt   10         0.007 ±       0.003    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.count                         thrpt   10        27.000                counts
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.time                          thrpt   10        32.000                    ms
    ```
    Consider that It tests the case of temporal typed UUID-like SimpleString interning/pooling, hence a pretty intensive case for the interner I've implemented because it need to compute hashCode and equals of long strings (~ 72 bytes).
    
    Some explanation:
    - score is the throughput in ops/sec
    - `artemisIntern` is the one using `SimpleString.Interner`
    - 'guavaInterner` is the one using the Guava Interner with weak References (the strong one is not faster TBH, probably a little slower)
    - the `3Threads` ones are testing 3 threads calling the interner concurrently
    
    The results are pretty clear: ~400 MB/sec of allocation rate vs 0 and a much higher (~ x2,5) throughput (although most of the time is spent into hashCode and equals computations).
    I hope to have shown better why I've designed the interner in the way I've done.



---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159997146
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    --- End diff --
    
    I will provide some results for sure to justify this implementation :)
    Anyway the guava one uses concurrent map hence it is very different...


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @michaelandrepearce @clebertsuconic 
    Here are extended results using the same garbage collector used on the broker (G1) + different string lengths:
    ```
    Benchmark                                                                         (length)   Mode  Cnt         Score          Error   Units
    SimpleStringInternerBenchmark.artemisIntern                                              8  thrpt   10  30348978.018 ±  2410863.818   ops/s
    SimpleStringInternerBenchmark.artemisIntern:·gc.alloc.rate                               8  thrpt   10        ≈ 10⁻⁴                 MB/sec
    SimpleStringInternerBenchmark.artemisIntern:·gc.alloc.rate.norm                          8  thrpt   10        ≈ 10⁻⁵                   B/op
    SimpleStringInternerBenchmark.artemisIntern:·gc.count                                    8  thrpt   10           ≈ 0                 counts
    SimpleStringInternerBenchmark.artemisIntern                                             16  thrpt   10  21618296.000 ±  1425149.201   ops/s
    SimpleStringInternerBenchmark.artemisIntern:·gc.alloc.rate                              16  thrpt   10        ≈ 10⁻⁴                 MB/sec
    SimpleStringInternerBenchmark.artemisIntern:·gc.alloc.rate.norm                         16  thrpt   10        ≈ 10⁻⁵                   B/op
    SimpleStringInternerBenchmark.artemisIntern:·gc.count                                   16  thrpt   10           ≈ 0                 counts
    SimpleStringInternerBenchmark.artemisIntern                                             36  thrpt   10  14261279.409 ±  1252627.430   ops/s
    SimpleStringInternerBenchmark.artemisIntern:·gc.alloc.rate                              36  thrpt   10        ≈ 10⁻⁴                 MB/sec
    SimpleStringInternerBenchmark.artemisIntern:·gc.alloc.rate.norm                         36  thrpt   10        ≈ 10⁻⁵                   B/op
    SimpleStringInternerBenchmark.artemisIntern:·gc.count                                   36  thrpt   10           ≈ 0                 counts
    SimpleStringInternerBenchmark.artemisIntern3Threads                                      8  thrpt   10  87213515.457 ±  5355377.340   ops/s
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.alloc.rate                       8  thrpt   10         0.004 ±        0.011  MB/sec
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.alloc.rate.norm                  8  thrpt   10        ≈ 10⁻⁴                   B/op
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.churn.G1_Eden_Space              8  thrpt   10         1.579 ±        7.551  MB/sec
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.churn.G1_Eden_Space.norm         8  thrpt   10         0.030 ±        0.143    B/op
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.count                            8  thrpt   10         1.000                 counts
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.time                             8  thrpt   10        14.000                     ms
    SimpleStringInternerBenchmark.artemisIntern3Threads                                     16  thrpt   10  64572878.031 ±  3119600.421   ops/s
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.alloc.rate                      16  thrpt   10         0.003 ±        0.005  MB/sec
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.alloc.rate.norm                 16  thrpt   10        ≈ 10⁻⁴                   B/op
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.churn.G1_Eden_Space             16  thrpt   10         1.598 ±        7.639  MB/sec
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.churn.G1_Eden_Space.norm        16  thrpt   10         0.041 ±        0.197    B/op
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.count                           16  thrpt   10         1.000                 counts
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.time                            16  thrpt   10         8.000                     ms
    SimpleStringInternerBenchmark.artemisIntern3Threads                                     36  thrpt   10  42088800.539 ±  2156595.581   ops/s
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.alloc.rate                      36  thrpt   10         0.005 ±        0.012  MB/sec
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.alloc.rate.norm                 36  thrpt   10        ≈ 10⁻⁴                   B/op
    SimpleStringInternerBenchmark.artemisIntern3Threads:·gc.count                           36  thrpt   10           ≈ 0                 counts
    SimpleStringInternerBenchmark.guavaInterner                                              8  thrpt   10  12371381.743 ±   352958.763   ops/s
    SimpleStringInternerBenchmark.guavaInterner:·gc.alloc.rate                               8  thrpt   10       440.284 ±       12.602  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.alloc.rate.norm                          8  thrpt   10        56.000 ±        0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Eden_Space                      8  thrpt   10       441.328 ±       74.932  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Eden_Space.norm                 8  thrpt   10        56.094 ±        8.844    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Old_Gen                         8  thrpt   10         0.001 ±        0.002  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Old_Gen.norm                    8  thrpt   10        ≈ 10⁻⁴                   B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.count                                    8  thrpt   10        46.000                 counts
    SimpleStringInternerBenchmark.guavaInterner:·gc.time                                     8  thrpt   10       125.000                     ms
    SimpleStringInternerBenchmark.guavaInterner                                             16  thrpt   10   9327233.046 ±   417047.080   ops/s
    SimpleStringInternerBenchmark.guavaInterner:·gc.alloc.rate                              16  thrpt   10       426.812 ±       19.133  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.alloc.rate.norm                         16  thrpt   10        72.000 ±        0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Eden_Space                     16  thrpt   10       431.758 ±       76.499  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Eden_Space.norm                16  thrpt   10        72.718 ±       10.613    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Old_Gen                        16  thrpt   10         0.002 ±        0.002  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Old_Gen.norm                   16  thrpt   10        ≈ 10⁻⁴                   B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.count                                   16  thrpt   10        45.000                 counts
    SimpleStringInternerBenchmark.guavaInterner:·gc.time                                    16  thrpt   10       134.000                     ms
    SimpleStringInternerBenchmark.guavaInterner                                             36  thrpt   10   6038631.034 ±   139674.264   ops/s
    SimpleStringInternerBenchmark.guavaInterner:·gc.alloc.rate                              36  thrpt   10       429.853 ±        9.912  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.alloc.rate.norm                         36  thrpt   10       112.000 ±        0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Eden_Space                     36  thrpt   10       441.364 ±       74.895  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Eden_Space.norm                36  thrpt   10       114.978 ±       19.021    B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Old_Gen                        36  thrpt   10         0.001 ±        0.002  MB/sec
    SimpleStringInternerBenchmark.guavaInterner:·gc.churn.G1_Old_Gen.norm                   36  thrpt   10        ≈ 10⁻³                   B/op
    SimpleStringInternerBenchmark.guavaInterner:·gc.count                                   36  thrpt   10        46.000                 counts
    SimpleStringInternerBenchmark.guavaInterner:·gc.time                                    36  thrpt   10       123.000                     ms
    SimpleStringInternerBenchmark.guavaInterner3Threads                                      8  thrpt   10  33995156.308 ±  1729710.281   ops/s
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.alloc.rate                       8  thrpt   10      1209.763 ±       61.810  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.alloc.rate.norm                  8  thrpt   10        56.000 ±        0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Eden_Space              8  thrpt   10      1218.226 ±       97.936  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Eden_Space.norm         8  thrpt   10        56.394 ±        3.577    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Old_Gen                 8  thrpt   10         0.005 ±        0.005  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Old_Gen.norm            8  thrpt   10        ≈ 10⁻⁴                   B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.count                            8  thrpt   10       127.000                 counts
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.time                             8  thrpt   10       313.000                     ms
    SimpleStringInternerBenchmark.guavaInterner3Threads                                     16  thrpt   10  25891674.274 ±  1803951.968   ops/s
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.alloc.rate                      16  thrpt   10      1184.541 ±       82.284  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.alloc.rate.norm                 16  thrpt   10        72.000 ±        0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Eden_Space             16  thrpt   10      1189.507 ±       74.552  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Eden_Space.norm        16  thrpt   10        72.348 ±        3.358    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Old_Gen                16  thrpt   10         0.006 ±        0.004  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Old_Gen.norm           16  thrpt   10        ≈ 10⁻³                   B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.count                           16  thrpt   10       124.000                 counts
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.time                            16  thrpt   10       320.000                     ms
    SimpleStringInternerBenchmark.guavaInterner3Threads                                     36  thrpt   10  16800145.225 ±   905870.505   ops/s
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.alloc.rate                      36  thrpt   10      1195.717 ±       64.515  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.alloc.rate.norm                 36  thrpt   10       112.001 ±        0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Eden_Space             36  thrpt   10      1198.855 ±       75.838  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Eden_Space.norm        36  thrpt   10       112.304 ±        4.281    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Old_Gen                36  thrpt   10         0.006 ±        0.004  MB/sec
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.churn.G1_Old_Gen.norm           36  thrpt   10         0.001 ±        0.001    B/op
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.count                           36  thrpt   10       125.000                 counts
    SimpleStringInternerBenchmark.guavaInterner3Threads:·gc.time                            36  thrpt   10       307.000                     ms
    ```



---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160002905
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    +
    +   private final T[] entries;
    --- End diff --
    
    If so yes you do, and you should avoid the word intern then, as typically expectation is similar behaviour to String.intern (which is what guava's does)


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160004310
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    > to check if equal/hash you're still having to process through that bytebuf bringing the bytes (if off heap) onto heap and doing a calc on them.
    
    That cost is reduced avoiding bounds checking and limiting the length of the byte stream and if compare very well vs byte[] copy + SimpleString allocation (measured and tested with JMH).
    
    >All creating SimpleString with data does is simple sets the byte[] as data, no decoding.
    
    It is missing a couple of parts: byte[] allocation and byte[] copy (from the ByteBuf).
    It seems sttrange, but it is avoiding the byte[] that is reduced by far most of the garbage



---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159977160
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    should keep method back compatible. 


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160005095
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    Ok, so thats simple to change, why don't simple make SimpleString hold data as ByteBuf not byte[].
    
    This then also would save this, and save further.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 closed the pull request at:

    https://github.com/apache/activemq-artemis/pull/1752


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160005087
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    --- End diff --
    
    Tomorrow (today in Italy :)) I will provide a JMH benchmark to show the difference between each approach (guava too), but consider the semantic differences between the interners: mine is packing all the pooled instances into a Object[] of fixed size not scattered along the heap and without using memory barriers to solve the potential races, while the Guava one is using a concurrent hash map with soft references, so the comparision won't be 100% fair.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160011547
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    --- End diff --
    
    And then configure the consumer also to use it


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159977324
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    --- End diff --
    
    Why not simply use Google's guava interners?


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160007968
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    +
    +   private final T[] entries;
    --- End diff --
    
    Yeah but notice how its named under package pool, because really its just pooling the x number of last Strings that the CharSequence has been toString'd.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159974168
  
    --- Diff: artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ActiveMQClientProtocolManager.java ---
    @@ -510,7 +510,7 @@ private void notifyTopologyChange(final ClusterTopologyChangeMessage topMessage)
           }
        }
     
    -   protected PacketDecoder getPacketDecoder() {
    +   protected PacketDecoder createPacketDecoder() {
    --- End diff --
    
    There's no need for this change.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159993138
  
    --- Diff: artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/ServerPacketDecoder.java ---
    @@ -83,16 +85,34 @@
     
     public class ServerPacketDecoder extends ClientPacketDecoder {
     
    +   private static final int UUID_LENGTH = 36;
    +   private static final int DEFAULT_INTERNER_CAPACITY = 32;
        private static final long serialVersionUID = 3348673114388400766L;
    -   public static final ServerPacketDecoder INSTANCE = new ServerPacketDecoder();
    +   private SimpleString.Interner keysInterner;
    +   private TypedProperties.StringValue.Interner valuesInterner;
     
    -   private static SessionSendMessage decodeSessionSendMessage(final ActiveMQBuffer in, CoreRemotingConnection connection) {
    +   public ServerPacketDecoder() {
    +      this.keysInterner = null;
    +      this.valuesInterner = null;
    +   }
    +
    +   private void initializeInternersIfNeeded() {
    --- End diff --
    
    If though you think about it address SimpleString really can be shared every where, if the aim is to achieve / avoid creation of SimpleString's on the heap where its the same address really, then this can be achieved now in java 8 easily with a normal interner and creating objects within functions.


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    Please do not merge it: I need to write down some results first and run all the CI tests :+1: 


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @franz1981 i noticed you dropped the committer history though. 


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159993639
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    its not, you removed a public method, from a very common class thats exposed to clients. 


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @michaelandrepearce That's perfect thanks :) Tomorrow I will take it from there then!


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @franz1981 if you merge if you're happy with suggested changes to your feature, then this PR would update.
    
    Re UseStringDeduplication i don't think it will help here where the aim is to avoid instance creation, as the strings will still be created, it only gets deduced on GC, as such if anything you may increase the GC workload.


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @franz1981 that should be used.


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160011036
  
    --- Diff: artemis-core-client/src/main/java/org/apache/activemq/artemis/core/message/impl/CoreMessage.java ---
    @@ -528,8 +543,17 @@ public void decodeHeadersAndProperties(final ByteBuf buffer) {
        private void decodeHeadersAndProperties(final ByteBuf buffer, boolean lazyProperties) {
           messageIDPosition = buffer.readerIndex();
           messageID = buffer.readLong();
    -
    -      address = SimpleString.readNullableSimpleString(buffer);
    +      int b = buffer.readByte();
    +      if (b != DataConstants.NULL) {
    +         final int length = buffer.readInt();
    +         if (keysInterner != null) {
    +            address = keysInterner.intern(buffer, length);
    --- End diff --
    
    btw, on another note, correlation id will kill you as each one should be unique, so would constantly invalidate, need to be a bit careful on property values to pool


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160003657
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    +
    +   private final T[] entries;
    --- End diff --
    
    fair enough: I've taken the name from here https://github.com/OpenHFT/Java-Lang/blob/master/lang/src/main/java/net/openhft/lang/pool/StringInterner.java
    And I've always thought is not such a bad name


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @clebertsuconic you're right , all good points: I was trying to understand which parts could benefit (if needed of course) such kind of optimizations too and @michaelandrepearce is helping me a lot to understand common cases that I haven't covered properly or that are breaking any of the assumptions I've made.
    Re the benefits the one I can anticipate with a standard JMS test is a reduction of >3X of garbage productions without any perf regression, but as @michaelandrepearce pointed me maybe other tests would invalidate such improvements...


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160003269
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    It is a github fault, but probably dependent by how i've formatted the code :P
    If you look at the real code you will see that the method is present and used :+1: 


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159996780
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    I'm missing something for sure: I've added a method using the interners, but not removed the old one...or not?


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r159994860
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    --- End diff --
    
    Guava's is fairly good tbh. As noted else where it is heavily used in other projects etc so also is much more well battle tested.


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to String ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    Thanks @michaelandrepearce, the work on https://github.com/franz1981/activemq-artemis/pull/4 seems fairly good to me, I've just added a couple of comments on the code!
    I would love the opinions of @clebertsuconic and @tabish121 too about it, in particular to how apply on AMQP or if there is anything we're missing :+1:
    
    Re the specific case of String I'm trying -XX:+UseStringDeduplication -XX:+PrintStringDeduplicationStatistics on Artemis to see if it could bring some benefits (for free) now that we're using G1 as the default broker collector, wdyt?


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @michaelandrepearce I've already addressed it! I will wait until the CI tests will finish to update this one :P
    Thanks!!!


---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @michaelandrepearce Yep I've messed up with the commits order while squashing: do you have any idea on how to restore it with ease?
    It makes sense to have just one commit, but I don't want to loose your contribution :)


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160020175
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/collections/TypedProperties.java ---
    @@ -329,7 +331,9 @@ public boolean containsProperty(final SimpleString key) {
           }
        }
     
    -   public synchronized void decode(final ByteBuf buffer) {
    --- End diff --
    
    haha


---

[GitHub] activemq-artemis pull request #1752: ARTEMIS-1586 Reduce GC pressure due to ...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1752#discussion_r160003612
  
    --- Diff: artemis-commons/src/main/java/org/apache/activemq/artemis/utils/AbstractInterner.java ---
    @@ -0,0 +1,157 @@
    +/**
    + * 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.activemq.artemis.utils;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.internal.MathUtil;
    +import io.netty.util.internal.PlatformDependent;
    +
    +/**
    + * Thread-safe {@code <T>} interner.
    + * <p>
    + * Differently from {@link String#intern()} it contains a fixed amount of entries and
    + * when used by concurrent threads it doesn't ensure the uniqueness of the entries ie
    + * the same entry could be allocated multiple times by concurrent calls.
    + */
    +public abstract class AbstractInterner<T> {
    --- End diff --
    
    btw for the guava implementation i did i ran this very rudimentary test (I am not saying in anyway this is a true perf test, but just to provide something indicative), so it run for a while and take final run results from each idea to let jit some chance to do what ever it needs, and the array to hold objects is to simulate the objects sticking around for a bit, and give some gc pressure.
    
    
    ```
      public static void main(String... args) throws InterruptedException {
          
          byte[] testString = "hello".getBytes();
    
          
          int size = 10000000;
    
    
          int inx = 0;
          long start2 = 0;
          long end2 = 0;
          long start = 0;
          long end = 0;
          SimpleString[] array = new SimpleString[size * 10];
    
          
          
          for (int k = 0; k < 5; k++) {
             array = new SimpleString[size * 10];
             inx = 0;
    
             System.gc();
             Thread.sleep(1000);
    
             for (int j = 0; j < 10; j++) {
                start = System.nanoTime();
                for (int i = 0; i < size; i++) {
                   SimpleString simpleString = new SimpleString(testString);
                   array[inx] = simpleString;
                   inx++;
                }
                end = System.nanoTime();
             }
    
             array = new SimpleString[size * 10];
             inx = 0;
    
             System.gc();
             Thread.sleep(1000);
             for (int j = 0; j < 10; j++) {
                start2 = System.nanoTime();
                for (int i = 0; i < size; i++) {
                   SimpleString simpleString = SimpleString.toSimpleString(testString);
                   array[inx] = simpleString;
                   inx++;
                }
                end2 = System.nanoTime();
             }
          }
    
          System.out.println("new      " + (end-start));
          System.out.println("intern   " + (end2-start2));
    
       }
    ```
    
    outputs:
    
    new      35102118957
    intern   368258702



---

[GitHub] activemq-artemis issue #1752: ARTEMIS-1586 Reduce GC pressure due to string ...

Posted by franz1981 <gi...@git.apache.org>.
Github user franz1981 commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1752
  
    @clebertsuconic Thanks to @michaelandrepearce that has provided a much more refined solution I'm running on CI the tests to see if it works as expected: I will provide soon some results


---