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/29 07:25:41 UTC

git commit: PHOENIX-1208 Check for existence of views doesn't take into account the fact that SYSTEM.CATALOG could be split across regions

Repository: phoenix
Updated Branches:
  refs/heads/4.0 13903e055 -> 6fb2b22b9


PHOENIX-1208 Check for existence of views doesn't take into account the fact that SYSTEM.CATALOG could be split across regions


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

Branch: refs/heads/4.0
Commit: 6fb2b22b9a30ecf74b608cc1d6081b7889763f20
Parents: 13903e0
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Aug 28 22:29:50 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Aug 28 22:29:50 2014 -0700

----------------------------------------------------------------------
 .../coprocessor/MetaDataEndpointImpl.java       | 73 ++++++++------------
 .../phoenix/coprocessor/SuffixFilter.java       | 50 ++++++++++++++
 2 files changed, 79 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6fb2b22b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 2f7b34f..020a3c9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -17,8 +17,6 @@
  */
 package org.apache.phoenix.coprocessor;
 
-import static com.google.common.collect.Lists.newArrayList;
-import static org.apache.hadoop.hbase.filter.CompareFilter.CompareOp.EQUAL;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME_INDEX;
@@ -73,15 +71,18 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
@@ -857,12 +858,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         locks.add(rowLock);
     }
     
-    private static final byte[] PHYSICAL_TABLE_BYTES = new byte[] {PTable.LinkType.PHYSICAL_TABLE.getSerializedValue()};
+    protected static final byte[] PHYSICAL_TABLE_BYTES = new byte[] {PTable.LinkType.PHYSICAL_TABLE.getSerializedValue()};
     /**
      * @param tableName parent table's name
      * @return true if there exist a table that use this table as their base table.
      * TODO: should we pass a timestamp here?
      */
+    @SuppressWarnings("deprecation")
     private boolean hasViews(HRegion region, byte[] tenantId, PTable table) throws IOException {
         byte[] schemaName = table.getSchemaName().getBytes();
         byte[] tableName = table.getTableName().getBytes();
@@ -877,21 +879,36 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             scan.setStartRow(startRow);
             scan.setStopRow(stopRow);
         }
-        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, EQUAL, PHYSICAL_TABLE_BYTES);
+        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, PHYSICAL_TABLE_BYTES);
         linkFilter.setFilterIfMissing(true);
         byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil.getTableNameAsBytes(schemaName, tableName));
         SuffixFilter rowFilter = new SuffixFilter(suffix);
         Filter filter = new FilterList(linkFilter, rowFilter);
         scan.setFilter(filter);
         scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
-        RegionScanner scanner = region.getScanner(scan);
+        // Original region-only scanner modified due to PHOENIX-1208
+        // RegionScanner scanner = region.getScanner(scan);
+        // The following *should* work, but doesn't due to HBASE-11837
+        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
+        // HTableInterface hTable = env.getTable(systemCatalogTableName);
+        // These deprecated calls work around the issue
+        HTablePool pool = new HTablePool (env.getConfiguration(),1);
         try {
-            List<Cell> results = newArrayList();
-            scanner.next(results);
-            return results.size() > 0;
-        }
-        finally {
-            scanner.close();
+            HTableInterface hTable = pool.getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
+            try {
+                ResultScanner scanner = hTable.getScanner(scan);
+                try {
+                    Result result = scanner.next();
+                    return result != null;
+                }
+                finally {
+                    scanner.close();
+                }
+            } finally {
+                hTable.close();
+            }
+        } finally {
+            pool.close();
         }
     }
     
@@ -1562,36 +1579,4 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return new MetaDataMutationResult(MutationCode.TABLE_NOT_IN_REGION,
                 EnvironmentEdgeManager.currentTimeMillis(), null);
     }
-
-    /**
-     * 
-     * Matches rows that end with a given byte array suffix
-     *
-     * 
-     * @since 3.0
-     */
-    private static class SuffixFilter extends FilterBase {
-        protected byte[] suffix = null;
-
-        public SuffixFilter(final byte[] suffix) {
-            this.suffix = suffix;
-        }
-        
-        @Override
-        public ReturnCode filterKeyValue(Cell ignored) throws IOException {
-          return ReturnCode.INCLUDE;
-        }
-        
-        @Override
-        public boolean filterRowKey(byte[] buffer, int offset, int length) {
-            if (buffer == null || this.suffix == null) return true;
-            if (length < suffix.length) return true;
-            // if they are equal, return false => pass row
-            // else return true, filter row
-            // if we are passed the suffix, set flag
-            int cmp = Bytes.compareTo(buffer, offset + (length - this.suffix.length),
-                    this.suffix.length, this.suffix, 0, this.suffix.length);
-            return cmp != 0;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6fb2b22b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SuffixFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SuffixFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SuffixFilter.java
new file mode 100644
index 0000000..e6bf9ac
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SuffixFilter.java
@@ -0,0 +1,50 @@
+package org.apache.phoenix.coprocessor;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * 
+ * Matches rows that end with a given byte array suffix
+ *
+ * 
+ * @since 3.0
+ */
+public class SuffixFilter extends FilterBase {
+    protected byte[] suffix = null;
+
+    public SuffixFilter(final byte[] suffix) {
+        this.suffix = suffix;
+    }
+    
+    @Override
+    public byte[] toByteArray() throws IOException {
+        return suffix;
+    }
+
+    @Override
+    public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+      return ReturnCode.INCLUDE;
+    }
+    
+    @Override
+    public boolean filterRowKey(byte[] buffer, int offset, int length) {
+        if (buffer == null || this.suffix == null) return true;
+        if (length < suffix.length) return true;
+        // if they are equal, return false => pass row
+        // else return true, filter row
+        // if we are passed the suffix, set flag
+        int cmp = Bytes.compareTo(buffer, offset + (length - this.suffix.length),
+                this.suffix.length, this.suffix, 0, this.suffix.length);
+        return cmp != 0;
+    }
+
+    
+    public static SuffixFilter parseFrom(final byte [] pbBytes) throws DeserializationException {
+        return new SuffixFilter(pbBytes);
+    }
+}
\ No newline at end of file