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 2014/08/10 03:56:21 UTC

[2/2] git commit: PHOENIX-1157 Improve abstraction for meta data cache

PHOENIX-1157 Improve abstraction for meta data cache

Conflicts:
	phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
	phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java


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

Branch: refs/heads/4.0
Commit: 64d136d15707bb5122ae1cfc00717c757086385e
Parents: d5a7803
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Aug 9 18:21:22 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Aug 9 18:27:51 2014 -0700

----------------------------------------------------------------------
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  6 +-
 .../query/ConnectionQueryServicesImpl.java      | 10 +--
 .../query/ConnectionlessQueryServicesImpl.java  |  4 +-
 .../query/DelegateConnectionQueryServices.java  |  4 +-
 .../apache/phoenix/query/MetaDataMutated.java   |  2 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  7 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    | 68 +++++++++++++-------
 .../phoenix/schema/PMetaDataImplTest.java       |  3 +-
 8 files changed, 64 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/64d136d1/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 364e61f..70f88f2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -726,10 +726,10 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
     }
 
     @Override
-    public PMetaData removeTable(PName tenantId, String tableName) throws SQLException {
-        metaData = metaData.removeTable(tenantId, tableName);
+    public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
+        metaData = metaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
         //Cascade through to connectionQueryServices too
-        getQueryServices().removeTable(tenantId, tableName);
+        getQueryServices().removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
         return metaData;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64d136d1/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 5065632..ee0be95 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
@@ -462,7 +462,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // and the next time it's used it'll be pulled over from the server.
                     if (waitTime <= 0) {
                         logger.warn("Unable to update meta data repo within " + (DEFAULT_OUT_OF_ORDER_MUTATIONS_WAIT_TIME_MS/1000) + " seconds for " + tableName);
-                        metaData = metaData.removeTable(tenantId, tableName);
+                        // There will never be a parentTableName here, as that would only
+                        // be non null for an index an we never add/remove columns from an index.
+                        metaData = metaData.removeTable(tenantId, tableName, null, HConstants.LATEST_TIMESTAMP);
                         break;
                     }
                     latestMetaDataLock.wait(waitTime);
@@ -493,10 +495,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
      }
 
     @Override
-    public PMetaData removeTable(PName tenantId, final String tableName) throws SQLException {
-        synchronized(latestMetaDataLock) {
+    public PMetaData removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
+        synchronized (latestMetaDataLock) {
             throwConnectionClosedIfNullMetaData();
-            latestMetaData = latestMetaData.removeTable(tenantId, tableName);
+            latestMetaData = latestMetaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
             latestMetaDataLock.notifyAll();
             return latestMetaData;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64d136d1/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 223abb6..ea121dd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -160,9 +160,9 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData removeTable(PName tenantId, String tableName)
+    public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp)
             throws SQLException {
-        return metaData = metaData.removeTable(tenantId, tableName);
+        return metaData = metaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64d136d1/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 3c119de..306d536 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -86,9 +86,9 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData removeTable(PName tenantId, String tableName)
+    public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp)
             throws SQLException {
-        return getDelegate().removeTable(tenantId, tableName);
+        return getDelegate().removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64d136d1/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
index 2e6da04..1b8ebda 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
@@ -35,7 +35,7 @@ import org.apache.phoenix.schema.PTable;
  */
 public interface MetaDataMutated {
     PMetaData addTable(PTable table) throws SQLException;
-    PMetaData removeTable(PName tenantId, String tableName) throws SQLException;
+    PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException;
     PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows) throws SQLException;
     PMetaData removeColumn(PName tenantId, String tableName, String familyName, String columnName, long tableTimeStamp, long tableSeqNum) throws SQLException;
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64d136d1/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 0e25449..cda956d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -343,7 +343,7 @@ public class MetaDataClient {
                         return result;
                     }
                     if (code == MutationCode.TABLE_NOT_FOUND && tryCount + 1 == maxTryCount) {
-                        connection.removeTable(tenantId, fullTableName);
+                        connection.removeTable(tenantId, fullTableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
                     }
                 }
             }
@@ -1460,7 +1460,7 @@ public class MetaDataClient {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
                         .setSchemaName(schemaName).setTableName(tableName).build().buildException();
                 default:
-                    connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, tableName));
+                    connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, tableName), parentTableName, result.getMutationTime());
                                         
                     if (result.getTable() != null && tableType != PTableType.VIEW) {
                         connection.setAutoCommit(true);
@@ -1527,7 +1527,8 @@ public class MetaDataClient {
         PName tenantId = connection.getTenantId();
         switch (mutationCode) {
         case TABLE_NOT_FOUND:
-            connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, tableName));
+            // Only called for add/remove column so parentTableName will always be null
+            connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, tableName), null, HConstants.LATEST_TIMESTAMP);
             throw new TableNotFoundException(schemaName, tableName);
         case UNALLOWED_TABLE_MUTATION:
             String columnName = null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64d136d1/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index dff0e40..8b26709 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.util.TimeKeeper;
 
 import com.google.common.collect.Lists;
@@ -316,41 +317,60 @@ public class PMetaDataImpl implements PMetaData {
     }
 
     @Override
-    public PMetaData removeTable(PName tenantId, String tableName) throws SQLException {
-        PTable table;
-        PTableCache tables = metaData.clone();
-        if ((table=tables.remove(new PTableKey(tenantId, tableName))) == null) {
-            return this;
+    public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
+        PTableCache tables = null;
+        PTableRef parentTableRef = null;
+        PTableKey key = new PTableKey(tenantId, tableName);
+        if (metaData.get(key) == null) {
+            if (parentTableName != null) {
+                parentTableRef = metaData.get(new PTableKey(tenantId, parentTableName));
+            }
+            if (parentTableRef == null) {
+                return this;
+            }
         } else {
+            tables = metaData.clone();
+            PTable table = tables.remove(key);
             for (PTable index : table.getIndexes()) {
                 tables.remove(index.getKey());
             }
-            // also remove its reference from parent table
-            PName parent = table.getParentName();
-            PTableRef parentTableRef = null;
-            if(parent != null && (parentTableRef=tables.get(new PTableKey(tenantId, parent.getString()))) != null) {
-                List<PTable> oldIndexes = parentTableRef.table.getIndexes();
-                if(oldIndexes != null && !oldIndexes.isEmpty()) {
-	                List<PTable> newIndexes = Lists.newArrayListWithExpectedSize(oldIndexes.size());
-	                newIndexes.addAll(oldIndexes);
-	                for (int i = 0; i < newIndexes.size(); i++) {
-	                    PTable index = newIndexes.get(i);
-	                    if (index.getName().equals(table.getName())) {
-	                        newIndexes.remove(i);
-	                        break;
-	                    }
-	                }
-	                PTable parentTable = PTableImpl.makePTable(parentTableRef.table, table.getTimeStamp(), newIndexes);
-	                tables.put(parentTable.getKey(), parentTable);
+            if (table.getParentName() != null) {
+                parentTableRef = tables.get(new PTableKey(tenantId, table.getParentName().getString()));
+            }
+        }
+        // also remove its reference from parent table
+        if (parentTableRef != null) {
+            List<PTable> oldIndexes = parentTableRef.table.getIndexes();
+            if(oldIndexes != null && !oldIndexes.isEmpty()) {
+                List<PTable> newIndexes = Lists.newArrayListWithExpectedSize(oldIndexes.size());
+                newIndexes.addAll(oldIndexes);
+                for (int i = 0; i < newIndexes.size(); i++) {
+                    PTable index = newIndexes.get(i);
+                    if (index.getName().getString().equals(tableName)) {
+                        newIndexes.remove(i);
+                        PTable parentTable = PTableImpl.makePTable(
+                                parentTableRef.table,
+                                tableTimeStamp == HConstants.LATEST_TIMESTAMP ? parentTableRef.table.getTimeStamp() : tableTimeStamp,
+                                newIndexes);
+                        if (tables == null) { 
+                            tables = metaData.clone();
+                        }
+                        tables.put(parentTable.getKey(), parentTable);
+                        break;
+                    }
                 }
             }
         }
-        return new PMetaDataImpl(tables);
+        return tables == null ? this : new PMetaDataImpl(tables);
     }
     
     @Override
     public PMetaData removeColumn(PName tenantId, String tableName, String familyName, String columnName, long tableTimeStamp, long tableSeqNum) throws SQLException {
-        PTable table = getTable(new PTableKey(tenantId, tableName));
+        PTableRef tableRef = metaData.get(new PTableKey(tenantId, tableName));
+        if (tableRef == null) {
+            return this;
+        }
+        PTable table = tableRef.table;
         PTableCache tables = metaData.clone();
         PColumn column;
         if (familyName == null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64d136d1/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
index 5584bac..9379ef3 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import java.sql.SQLException;
 import java.util.Set;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.util.TimeKeeper;
 import org.junit.Test;
 
@@ -35,7 +36,7 @@ public class PMetaDataImplTest {
     }
     
     private static PMetaData removeFromTable(PMetaData metaData, String name) throws SQLException {
-        return metaData.removeTable(null, name);
+        return metaData.removeTable(null, name, null, HConstants.LATEST_TIMESTAMP);
     }
     
     private static PTable getFromTable(PMetaData metaData, String name) throws TableNotFoundException {