You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/03/14 18:19:02 UTC

phoenix git commit: PHOENIX-1677 Immutable index deadlocks when number of guideposts are one half of thread pool size

Repository: phoenix
Updated Branches:
  refs/heads/master e94128881 -> 1c14dd192


PHOENIX-1677 Immutable index deadlocks when number of guideposts are one half of thread pool size


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

Branch: refs/heads/master
Commit: 1c14dd192530b108a8b6d81dcff8cd7032acbe3f
Parents: e941288
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Mar 14 10:18:57 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Mar 14 10:18:57 2015 -0700

----------------------------------------------------------------------
 .../index/ImmutableIndexWithStatsIT.java        | 90 ++++++++++++++++++++
 .../query/ConnectionQueryServicesImpl.java      |  2 +-
 .../org/apache/phoenix/query/HTableFactory.java |  3 +-
 3 files changed, 93 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c14dd19/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexWithStatsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexWithStatsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexWithStatsIT.java
new file mode 100644
index 0000000..e5b38f3
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexWithStatsIT.java
@@ -0,0 +1,90 @@
+/*
+ * 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.phoenix.end2end.index;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.Shadower;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+
+public class ImmutableIndexWithStatsIT extends BaseHBaseManagedTimeIT {
+    
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(5);
+        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(1));
+        props.put(QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, Boolean.TRUE.toString());
+        props.put(QueryServices.THREAD_POOL_SIZE_ATTRIB, Integer.toString(4));
+        props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(500));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+   
+    @Test
+    public void testIndexCreationDeadlockWithStats() throws Exception {
+        String query;
+        ResultSet rs;
+        
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        conn.createStatement().execute("CREATE TABLE " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR) IMMUTABLE_ROWS=TRUE");
+        query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME;
+        rs = conn.createStatement().executeQuery(query);
+        assertFalse(rs.next());
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " VALUES(?,?)");
+        for (int i=0; i<6;i++) {
+	        stmt.setString(1,"k" + i);
+	        stmt.setString(2, "v" + i );
+	        stmt.execute();
+        }
+        conn.commit();
+        
+        conn.createStatement().execute("UPDATE STATISTICS " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME);
+        query = "SELECT COUNT(*) FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME;
+        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+        assertTrue(QueryUtil.getExplainPlan(rs).startsWith("CLIENT 7-CHUNK PARALLEL 1-WAY FULL SCAN"));
+
+        conn.createStatement().execute("CREATE INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " (v)");
+        
+        query = "SELECT * FROM " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME;
+        rs = conn.createStatement().executeQuery(query);
+        assertTrue(rs.next());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c14dd19/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 d6f844c..49c946a 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
@@ -290,7 +290,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     @Override
     public HTableInterface getTable(byte[] tableName) throws SQLException {
         try {
-            return HBaseFactoryProvider.getHTableFactory().getTable(tableName, connection, getExecutor());
+            return HBaseFactoryProvider.getHTableFactory().getTable(tableName, connection, null);
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
             byte[][] schemaAndTableName = new byte[2][];
             SchemaUtil.getVarChars(tableName, schemaAndTableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1c14dd19/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java
index 7a10683..09dbff6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java
@@ -47,7 +47,8 @@ public interface HTableFactory {
     static class HTableFactoryImpl implements HTableFactory {
         @Override
         public HTableInterface getTable(byte[] tableName, HConnection connection, ExecutorService pool) throws IOException {
-            return connection.getTable(tableName, pool);
+            // Let the HBase client manage the thread pool instead of passing ours through
+            return connection.getTable(tableName);
         }
     }
 }