You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dp...@apache.org on 2018/05/21 14:28:39 UTC

ignite git commit: IGNITE-8469 Fix for non-heap memory leak for calling cluster activation multi times. - Fixes #3986.

Repository: ignite
Updated Branches:
  refs/heads/master eb8038f65 -> 9a4a145be


IGNITE-8469 Fix for non-heap memory leak for calling cluster activation multi times. - Fixes #3986.

Signed-off-by: dpavlov <dp...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/9a4a145b
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/9a4a145b
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/9a4a145b

Branch: refs/heads/master
Commit: 9a4a145be514e650258715a7e682d427d5812d16
Parents: eb8038f
Author: Maxim Muzafarov <ma...@gmail.com>
Authored: Mon May 21 17:28:22 2018 +0300
Committer: dpavlov <dp...@apache.org>
Committed: Mon May 21 17:28:22 2018 +0300

----------------------------------------------------------------------
 .../mem/file/MappedFileMemoryProvider.java      |  8 ++
 .../mem/unsafe/UnsafeMemoryProvider.java        |  8 ++
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |  3 +-
 .../apache/ignite/internal/util/GridDebug.java  | 56 +++++++++----
 .../pagemem/PageMemoryNoStoreLeakTest.java      | 82 ++++++++++++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite.java   |  2 +
 6 files changed, 144 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9a4a145b/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java
index 7186b27..54b4af4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java
@@ -55,6 +55,9 @@ public class MappedFileMemoryProvider implements DirectMemoryProvider {
     /** */
     private List<MappedFile> mappedFiles;
 
+    /** Flag shows if current memory provider have been already initialized. */
+    private boolean isInit;
+
     /**
      * @param allocationPath Allocation path.
      */
@@ -65,6 +68,9 @@ public class MappedFileMemoryProvider implements DirectMemoryProvider {
 
     /** {@inheritDoc} */
     @Override public void initialize(long[] sizes) {
+        if (isInit)
+            throw new IgniteException("Second initialization does not allowed for current provider");
+
         this.sizes = sizes;
 
         mappedFiles = new ArrayList<>(sizes.length);
@@ -90,6 +96,8 @@ public class MappedFileMemoryProvider implements DirectMemoryProvider {
                         "opened by another process and current user has enough rights): " + file);
             }
         }
+
+        isInit = true;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a4a145b/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java
index 276e10e..7eb58cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/mem/unsafe/UnsafeMemoryProvider.java
@@ -41,6 +41,9 @@ public class UnsafeMemoryProvider implements DirectMemoryProvider {
     /** */
     private IgniteLogger log;
 
+    /** Flag shows if current memory provider have been already initialized. */
+    private boolean isInit;
+
     /**
      * @param log Ignite logger to use.
      */
@@ -50,9 +53,14 @@ public class UnsafeMemoryProvider implements DirectMemoryProvider {
 
     /** {@inheritDoc} */
     @Override public void initialize(long[] sizes) {
+        if (isInit)
+            throw new IgniteException("Second initialization does not allowed for current provider");
+
         this.sizes = sizes;
 
         regions = new ArrayList<>();
+
+        isInit = true;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a4a145b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
index d4b22a6..54b647a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
@@ -231,7 +231,8 @@ public class PageMemoryNoStoreImpl implements PageMemory {
         if (lastIdx != SEG_CNT - 1)
             chunks = Arrays.copyOf(chunks, lastIdx + 1);
 
-        directMemoryProvider.initialize(chunks);
+        if (segments == null)
+            directMemoryProvider.initialize(chunks);
 
         addSegment(null);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a4a145b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
index 2fa148e..d26e060 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util;
 
 import com.sun.management.HotSpotDiagnosticMXBean;
+import com.sun.management.OperatingSystemMXBean;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
@@ -34,6 +35,7 @@ import java.util.Date;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.management.MBeanServer;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -75,6 +77,12 @@ public class GridDebug {
     /** field to store the hotspot diagnostic MBean */
     private static volatile HotSpotDiagnosticMXBean hotspotMBean;
 
+    /** Platform-specific management interface for the operating system. */
+    private static final String OS_BEAN_NAME = "java.lang:type=OperatingSystem";
+
+    /** Call to {@link #initOSMBean()} before accessing. */
+    private static volatile OperatingSystemMXBean osMBean;
+
     /* */
     static {
         if (LOGS_PATH != null) {
@@ -339,33 +347,53 @@ public class GridDebug {
     }
 
     /**
-     * Initialize the hotspot diagnostic MBean field
+     * @return Committed VM size in bits.
+     */
+    public static long getCommittedVirtualMemorySize() {
+        initOSMBean();
+
+        return osMBean.getCommittedVirtualMemorySize();
+    }
+
+    /**
+     * Initialize the hotspot diagnostic MBean field.
      */
     private static void initHotspotMBean() {
         if (hotspotMBean == null) {
             synchronized (GridDebug.class) {
                 if (hotspotMBean == null)
-                    hotspotMBean = getHotspotMBean();
+                    hotspotMBean = getMBean(HOTSPOT_BEAN_NAME, HotSpotDiagnosticMXBean.class);
             }
         }
     }
 
     /**
-     * Gets the hotspot diagnostic MBean from the platform MBean server
-     * @return Diagnostic bean.
+     * Initialize field to store OperatingSystem MXBean.
      */
-    private static HotSpotDiagnosticMXBean getHotspotMBean() {
-        try {
-            MBeanServer server = ManagementFactory.getPlatformMBeanServer();
+    private static void initOSMBean() {
+        if (osMBean == null) {
+            synchronized (GridDebug.class) {
+                if (osMBean == null)
+                    osMBean = getMBean(OS_BEAN_NAME, OperatingSystemMXBean.class);
+            }
+        }
+    }
 
-            HotSpotDiagnosticMXBean bean = ManagementFactory.newPlatformMXBeanProxy(server,
-                HOTSPOT_BEAN_NAME, HotSpotDiagnosticMXBean.class);
+    /**
+     * Get MXBean from the platform MBeanServer.
+     *
+     * @param mxbeanName The name for uniquely identifying the MXBean within an MBeanServer.
+     * @param mxbeanItf The MXBean interface.
+     * @return A proxy for a platform MXBean interface.
+     */
+    private static <T> T getMBean(String mxbeanName, Class<T> mxbeanItf) {
+        try {
+            MBeanServer srv = ManagementFactory.getPlatformMBeanServer();
 
-            return bean;
-        } catch (RuntimeException re) {
-            throw re;
-        } catch (Exception exp) {
-            throw new RuntimeException(exp);
+            return ManagementFactory.newPlatformMXBeanProxy(srv, mxbeanName, mxbeanItf);
+        }
+        catch (IOException e) {
+            throw new IgniteException(e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a4a145b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryNoStoreLeakTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryNoStoreLeakTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryNoStoreLeakTest.java
new file mode 100644
index 0000000..65e8c36
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryNoStoreLeakTest.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.pagemem;
+
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.internal.mem.DirectMemoryProvider;
+import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
+import org.apache.ignite.internal.util.typedef.internal.D;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Base scenario for memory leak:
+ * 1. Start topology with client nodes;
+ * 2. Call active(true) for cluster. This activation should fail by some circumstances (e.g. some locks exists);
+ * 3. IgniteCacheDatabaseSharedManager started and onActive called here. Memory allocated;
+ * 4. Call active(true) again. Activation successfull, non heap memory leak introduced;
+ */
+public class PageMemoryNoStoreLeakTest extends GridCommonAbstractTest {
+    /** */
+    private static final int PAGE_SIZE = 4 * 1024;
+
+    /** */
+    private static final int MAX_MEMORY_SIZE = 10 * 1024 * 1024;
+
+    /** Allow delta between GC executions. */
+    private static final int ALLOWED_DELTA = 10 * 1024 * 1024;
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageDoubleInitMemoryLeak() throws Exception {
+        long initVMsize = D.getCommittedVirtualMemorySize();
+
+        for (int i = 0; i < 1_000; i++) {
+            final DirectMemoryProvider provider = new UnsafeMemoryProvider(log());
+
+            final DataRegionConfiguration plcCfg = new DataRegionConfiguration()
+                .setMaxSize(MAX_MEMORY_SIZE).setInitialSize(MAX_MEMORY_SIZE);
+
+            PageMemory mem = new PageMemoryNoStoreImpl(
+                log(),
+                provider,
+                null,
+                PAGE_SIZE,
+                plcCfg,
+                new DataRegionMetricsImpl(plcCfg),
+                true);
+
+            try {
+                mem.start();
+
+                //Second initialization, introduces leak
+                mem.start();
+            }
+            finally {
+                mem.stop();
+            }
+
+            long committedVMSize = D.getCommittedVirtualMemorySize();
+
+            assertTrue(committedVMSize - initVMsize <= ALLOWED_DELTA);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a4a145b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
index ceb0f0d..7294a60 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.FillFacto
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.IndexStoragePageMemoryImplTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImplNoLoadTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImplTest;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryNoStoreLeakTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PagesWriteThrottleSmokeTest;
 import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBufferTest;
 import org.apache.ignite.internal.processors.database.IgniteDbDynamicCacheSelfTest;
@@ -61,6 +62,7 @@ public class IgnitePdsTestSuite extends TestSuite {
 
         // Basic PageMemory tests.
         suite.addTestSuite(PageMemoryImplNoLoadTest.class);
+        suite.addTestSuite(PageMemoryNoStoreLeakTest.class);
         suite.addTestSuite(IndexStoragePageMemoryImplTest.class);
         suite.addTestSuite(PageMemoryImplTest.class);