You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2019/04/17 08:55:54 UTC

[ignite] branch master updated: IGNITE-11755: Memory leak H2 connections at the ConnectionManager#detachedConns. This closes #6457.

This is an automated email from the ASF dual-hosted git repository.

amashenkov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 463ab8f  IGNITE-11755: Memory leak H2 connections at the ConnectionManager#detachedConns. This closes #6457.
463ab8f is described below

commit 463ab8f63eacb27a08eb3fa638b54f7a226524af
Author: tledkov <tl...@gridgain.com>
AuthorDate: Wed Apr 17 11:55:28 2019 +0300

    IGNITE-11755: Memory leak H2 connections at the ConnectionManager#detachedConns. This closes #6457.
    
    Signed-off-by: Andrey V. Mashenkov <an...@gmail.com>
---
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java      |   2 +
 .../thin/JdbcThinTransactionsLeaksMvccTest.java    | 150 +++++++++++++++++++++
 .../processors/query/h2/ConnectionManager.java     |   4 +-
 3 files changed, 155 insertions(+), 1 deletion(-)

diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index e77dfc9..32ae2d5 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -88,6 +88,7 @@ import org.apache.ignite.jdbc.thin.JdbcThinStreamingOrderedSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTcpIoTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsClientAutoCommitComplexSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsClientNoAutoCommitComplexSelfTest;
+import org.apache.ignite.jdbc.thin.JdbcThinTransactionsLeaksMvccTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsServerAutoCommitComplexSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsServerNoAutoCommitComplexSelfTest;
@@ -230,6 +231,7 @@ import org.junit.runners.Suite;
     JdbcThinAuthenticateConnectionSelfTest.class,
 
     JdbcThinPreparedStatementLeakTest.class,
+    JdbcThinTransactionsLeaksMvccTest.class,
 })
 public class IgniteJdbcDriverTestSuite {
 }
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsLeaksMvccTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsLeaksMvccTest.java
new file mode 100644
index 0000000..93818a5
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsLeaksMvccTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.jdbc.thin;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.query.NestedTxMode;
+import org.apache.ignite.internal.processors.query.h2.ConnectionManager;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.junit.Test;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_MAX_COMPLETED_TX_COUNT;
+
+/**
+ * Tests to check leaks at the ConnectionManager#detachedConns map.
+ */
+public class JdbcThinTransactionsLeaksMvccTest extends JdbcThinAbstractSelfTest {
+    /** */
+    private static final String URL = "jdbc:ignite:thin://127.0.0.1";
+
+    /** Keys count. */
+    private static final int KEYS = 10;
+
+    /** Iterations count. */
+    private static final int ITERATIONS = 1_000;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setSystemWorkerBlockedTimeout(Long.MAX_VALUE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGrids(3);
+
+        try (Connection c = c(true, NestedTxMode.ERROR)) {
+            try (Statement s = c.createStatement()) {
+                s.execute("CREATE TABLE TEST (k int primary key, v int) WITH \"atomicity=transactional_snapshot\"");
+
+                for (int i = 0; i < KEYS; ++i)
+                    s.execute("INSERT INTO TEST VALUES (" + i +", " + i + ")");
+
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @param autoCommit Auto commit mode.
+     * @param nestedTxMode Nested transactions mode.
+     * @return Connection.
+     * @throws SQLException if failed.
+     */
+    private static Connection c(boolean autoCommit, NestedTxMode nestedTxMode) throws SQLException {
+        Connection res = DriverManager.getConnection(URL + "/?nestedTransactionsMode=" + nestedTxMode.name());
+
+        res.setAutoCommit(autoCommit);
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    @Test
+    @WithSystemProperty(key=IGNITE_MAX_COMPLETED_TX_COUNT, value = "1024")
+    public void testLeaks() {
+        runQueries(ITERATIONS);
+
+        int prevDetachedConns = detachedConnectionCount(grid(0));
+
+        runQueries(ITERATIONS * 2);
+
+        int curDetachedConns = detachedConnectionCount(grid(0));
+
+        assertTrue("Detached connection leaks: prevSize=" + prevDetachedConns + ", curSize=" + curDetachedConns,
+            curDetachedConns < prevDetachedConns * 2 + 1);
+    }
+
+    /**
+     * @param iters Count of queries.
+     */
+    private void runQueries(int iters) {
+        for (int i = 0; i < iters; ++i) {
+            try (Connection c = c(false, NestedTxMode.ERROR)) {
+                try (Statement s = c.createStatement()) {
+                    s.execute("BEGIN");
+
+                    s.execute("SELECT * FROM TEST");
+
+                    ResultSet rs = s.getResultSet();
+
+                    int cnt = 0;
+
+                    while (rs.next())
+                        ++cnt;
+
+                    assertEquals(KEYS, cnt);
+
+                    c.commit();
+                }
+            }
+            catch (SQLException e) {
+                throw new AssertionError(e);
+            }
+        }
+    }
+
+    /**
+     * @param igx Node.
+     * @return Count of detached connections.
+     */
+    private int detachedConnectionCount(IgniteEx igx) {
+        ConnectionManager connMgr = ((IgniteH2Indexing)igx.context().query().getIndexing()).connections();
+
+        ConcurrentMap detachedConns = GridTestUtils.getFieldValue(connMgr, "detachedConns");
+
+        return detachedConns.size();
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
index 2b3776c..05e37a0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
@@ -32,8 +32,8 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine;
-import org.apache.ignite.internal.processors.query.h2.opt.H2PlainRowFactory;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2PlainRowFactory;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -465,6 +465,8 @@ public class ConnectionManager {
         }
 
         setConn.put(conn, false);
+
+        detachedConns.remove(conn);
     }
 
     /**