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