You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2015/10/16 02:13:13 UTC

phoenix git commit: PHOENIX-2284 Remove call to returnAllSequences when connection count drop to zero

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 c833ab8cc -> 329b16df8


PHOENIX-2284 Remove call to returnAllSequences when connection count drop to zero


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/329b16df
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/329b16df
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/329b16df

Branch: refs/heads/4.x-HBase-0.98
Commit: 329b16df8ff287d5a02fda5828e32eac757199a1
Parents: c833ab8
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Oct 15 17:12:42 2015 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Thu Oct 15 17:13:08 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/SequenceIT.java  | 24 +++++++++++
 .../query/ConnectionQueryServicesImpl.java      | 42 +++++++++++---------
 .../org/apache/phoenix/query/QueryServices.java |  1 +
 .../phoenix/query/QueryServicesOptions.java     |  2 +
 4 files changed, 51 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/329b16df/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
index 78f8132..3e7ec31 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
@@ -1302,5 +1302,29 @@ public class SequenceIT extends BaseClientManagedTimeIT {
         assertEquals(2, rs.getInt(1));
         assertEquals(5, rs.getInt(2));        
     }
+    
+    @Test
+    public void testReturnAllSequencesNotCalledForNoOpenConnections() throws Exception {
+        nextConnection();
+        conn.createStatement().execute("CREATE SEQUENCE alpha.zeta START WITH 3 INCREMENT BY 2 CACHE 5");
+        nextConnection();
+        String query = "SELECT NEXT VALUE FOR alpha.zeta FROM SYSTEM.\"SEQUENCE\"";
+        ResultSet rs = conn.prepareStatement(query).executeQuery();
+        assertTrue(rs.next());
+        assertEquals(3, rs.getInt(1));
+        assertFalse(rs.next());
+        rs = conn.prepareStatement(query).executeQuery();
+        assertTrue(rs.next());
+        assertEquals(5, rs.getInt(1));
+        assertFalse(rs.next());
+        conn.close();
+        
+        // verify that calling close() does not return sequence values back to the server
+        query = "SELECT CURRENT_VALUE FROM SYSTEM.\"SEQUENCE\" WHERE SEQUENCE_SCHEMA='ALPHA' AND SEQUENCE_NAME='ZETA'";
+        rs = conn.prepareStatement(query).executeQuery();
+        assertTrue(rs.next());
+        assertEquals(13, rs.getInt(1));
+        assertFalse(rs.next());
+    }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/329b16df/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index af51200..bd21f20 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -202,6 +202,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     @GuardedBy("connectionCountLock")
     private int connectionCount = 0;
     private final Object connectionCountLock = new Object();
+    private final boolean returnSequenceValues ;
 
     private HConnection connection;
     private volatile boolean initialized;
@@ -278,6 +279,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 .maximumSize(MAX_TABLE_STATS_CACHE_ENTRIES)
                 .expireAfterWrite(halfStatsUpdateFreq, TimeUnit.MILLISECONDS)
                 .build();
+        this.returnSequenceValues = config.getBoolean(QueryServices.RETURN_SEQUENCE_VALUES_ATTRIB, QueryServicesOptions.DEFAULT_RETURN_SEQUENCE_VALUES);
     }
 
     private void openConnection() throws SQLException {
@@ -2583,28 +2585,32 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     @Override
     public void addConnection(PhoenixConnection connection) throws SQLException {
-        synchronized (connectionCountLock) {
-            connectionCount++;
-        }
+    	if (returnSequenceValues) {
+	        synchronized (connectionCountLock) {
+	            connectionCount++;
+	        }
+    	}
     }
 
     @Override
     public void removeConnection(PhoenixConnection connection) throws SQLException {
-        ConcurrentMap<SequenceKey,Sequence> formerSequenceMap = null;
-        synchronized (connectionCountLock) {
-            if (--connectionCount == 0) {
-                if (!this.sequenceMap.isEmpty()) {
-                    formerSequenceMap = this.sequenceMap;
-                    this.sequenceMap = Maps.newConcurrentMap();
-                }
-            }
-        }
-        // Since we're using the former sequenceMap, we can do this outside
-        // the lock.
-        if (formerSequenceMap != null) {
-            // When there are no more connections, attempt to return any sequences
-            returnAllSequences(formerSequenceMap);
-        }
+    	if (returnSequenceValues) {
+	        ConcurrentMap<SequenceKey,Sequence> formerSequenceMap = null;
+	        synchronized (connectionCountLock) {
+	            if (--connectionCount == 0) {
+	                if (!this.sequenceMap.isEmpty()) {
+	                    formerSequenceMap = this.sequenceMap;
+	                    this.sequenceMap = Maps.newConcurrentMap();
+	                }
+	            }
+	        }
+	        // Since we're using the former sequenceMap, we can do this outside
+	        // the lock.
+	        if (formerSequenceMap != null) {
+	            // When there are no more connections, attempt to return any sequences
+	            returnAllSequences(formerSequenceMap);
+	        }
+    	}
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/329b16df/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 90e2051..835f639 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -167,6 +167,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB = "phoenix.functions.allowUserDefinedFunctions";
     public static final String COLLECT_REQUEST_LEVEL_METRICS = "phoenix.query.request.metrics.enabled";
     public static final String ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE = "phoenix.schema.view.newcf";
+    public static final String RETURN_SEQUENCE_VALUES_ATTRIB = "phoenix.sequence.returnValues";
     
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/329b16df/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 4dce467..9f69c05 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -196,6 +196,8 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_ALLOW_USER_DEFINED_FUNCTIONS = false;
     public static final boolean DEFAULT_REQUEST_LEVEL_METRICS_ENABLED = false;
     public static final boolean DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE = true;
+    
+    public static final boolean DEFAULT_RETURN_SEQUENCE_VALUES = false;
 
     private final Configuration config;