[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-19 Thread twdsilva
Github user twdsilva closed the pull request at:

https://github.com/apache/phoenix/pull/303


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-14 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r202522040
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java ---
@@ -231,36 +231,26 @@ public Object getPTableValue(PTable table) {
 private final SQLExceptionCode mutatingImmutablePropException;
 private final boolean isValidOnView;
 private final boolean isMutableOnView;
-private final boolean propagateToViews;
 
 private TableProperty(String propertyName, boolean isMutable, boolean 
isValidOnView, boolean isMutableOnView) {
-this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, true);
-}
-
-private TableProperty(String propertyName, boolean isMutable, boolean 
isValidOnView, boolean isMutableOnView, boolean propagateToViews) {
-this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, 
propagateToViews);
+this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView);
 }
 
 private TableProperty(String propertyName, SQLExceptionCode 
colFamilySpecifiedException, boolean isMutable, boolean isValidOnView, boolean 
isMutableOnView) {
-this(propertyName, colFamilySpecifiedException, isMutable, 
CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, true);
+this(propertyName, colFamilySpecifiedException, isMutable, 
CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView);
 }
 
 private TableProperty(String propertyName, boolean isMutable, boolean 
isValidOnView, boolean isMutableOnView, SQLExceptionCode isMutatingException) {
-this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, isMutatingException, isValidOnView, isMutableOnView, true);
+this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, isMutatingException, isValidOnView, isMutableOnView);
 }
 
 private TableProperty(String propertyName, SQLExceptionCode 
colFamSpecifiedException, boolean isMutable, SQLExceptionCode 
mutatingException, boolean isValidOnView, boolean isMutableOnView) {
-this(propertyName, colFamSpecifiedException, isMutable, 
mutatingException, isValidOnView, isMutableOnView, true);
-}
-
-private TableProperty(String propertyName, SQLExceptionCode 
colFamSpecifiedException, boolean isMutable, SQLExceptionCode 
mutatingException, boolean isValidOnView, boolean isMutableOnView, boolean 
propagateToViews) {
--- End diff --

I filed PHOENIX-4763 to fix this. We should be able to use the cell 
timestamp to differentiate, still need to figure out how to expose this since 
its the properties in PTable don't currently expose the timestamp. 


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201835494
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -372,6 +378,31 @@ public void testViewAndTableAndDrop() throws Exception 
{
 // drop table cascade should succeed
 conn.createStatement().execute("DROP TABLE " + fullTableName + " 
CASCADE");
 
+validateViewDoesNotExist(conn, fullViewName1);
+validateViewDoesNotExist(conn, fullViewName2);
+
+}
+
+@Test
+public void testRecreateDroppedTableWithChildViews() throws Exception {
--- End diff --

We write the parent->child link first, then if the table uses column 
encoding we update the encoded column qualifiers on the parent table, and 
finally use mutateRowsWithLocks to write the view metadata atomically. 
We ignore views that can't be found (in case writing the child view 
metadata fails). 
If the metadata write fails and the table uses column encoding then we will 
lose a few column qualifiers. 
I'll add a test for this.



---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201800952
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java ---
@@ -1893,26 +1981,45 @@ public static void upgradeTable(PhoenixConnection 
conn, String srcTable) throws
 conn.commit();
 }
 conn.getQueryServices().clearTableFromCache(
-conn.getTenantId() == null ? 
ByteUtil.EMPTY_BYTE_ARRAY : conn.getTenantId().getBytes(),
+tenantIdBytes,
 index.getSchemaName().getBytes(), 
index.getTableName().getBytes(),
 PhoenixRuntime.getCurrentScn(readOnlyProps));
 }
 updateIndexesSequenceIfPresent(conn, table);
 conn.commit();
-
 } else {
 throw new RuntimeException("Error: problem occured during 
upgrade. Table is not upgraded successfully");
 }
 if (table.getType() == PTableType.VIEW) {
 logger.info(String.format("Updating link information for 
view '%s' ..", table.getTableName()));
 updateLink(conn, oldPhysicalName, 
newPhysicalTablename,table.getSchemaName(),table.getTableName());
 conn.commit();
-
+
+// if the view is a first level child, then we need to 
create the PARENT_TABLE link
+// that was overwritten by the PHYSICAL_TABLE link 
--- End diff --

Yes this will make it so that the parent link row will be created correctly 
when upgrading tables to be namespace mapped. 


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201799117
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1779,13 +2174,65 @@ public void createTable(RpcController controller, 
CreateTableRequest request,
 }
 }
 
+// The mutations to create a table are written in the 
following order:
+// 1. Write the child link as if the next two steps fail we
+// ignore missing children while processing a parent
+// 2. Update the encoded column qualifier for the parent 
table if its on a
+// different region server (for tables that use column 
qualifier encoding)
+// if the next step fails we end up wasting a few col 
qualifiers
+// 3. Finally write the mutations to create the table
+
+// From 4.15 the parent->child links are stored in a 
separate table SYSTEM.CHILD_LINK
+List childLinkMutations = 
MetaDataUtil.removeChildLinks(tableMetadata);
--- End diff --

I filed PHOENIX-4810 and added a comment to reference this jira.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201776288
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -447,7 +447,7 @@
 static {
 Collections.sort(FUNCTION_KV_COLUMNS, KeyValue.COMPARATOR);
 }
-
+
--- End diff --

I modified the class level comment.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201776204
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1809,180 +2256,97 @@ public void createTable(RpcController controller, 
CreateTableRequest request,
 } catch (Throwable t) {
 logger.error("createTable failed", t);
 ProtobufUtil.setControllerException(controller,
-
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
+ServerUtil.createIOException(fullTableName, t));
 }
 }
 
+   private void dropChildMetadata(byte[] schemaName, byte[] tableName, 
byte[] tenantIdBytes)
+   throws IOException, SQLException, 
ClassNotFoundException {
+   TableViewFinderResult childViewsResult = new 
TableViewFinderResult();
+   findAllChildViews(tenantIdBytes, schemaName, tableName, 
childViewsResult);
+   if (childViewsResult.hasViews()) {
+   for (TableInfo viewInfo : 
childViewsResult.getResults()) {
+   byte[] viewTenantId = viewInfo.getTenantId();
+   byte[] viewSchemaName = 
viewInfo.getSchemaName();
+   byte[] viewName = viewInfo.getTableName();
+   Properties props = new Properties();
+   if (viewTenantId != null && viewTenantId.length 
!= 0)
+   
props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, 
Bytes.toString(viewTenantId));
+   try (PhoenixConnection connection = 
QueryUtil.getConnectionOnServer(env.getConfiguration())
+   
.unwrap(PhoenixConnection.class)) {
+   MetaDataClient client = new 
MetaDataClient(connection);
+   org.apache.phoenix.parse.TableName 
viewTableName = org.apache.phoenix.parse.TableName
+   
.create(Bytes.toString(viewSchemaName), Bytes.toString(viewName));
+   client.dropTable(
+   new 
DropTableStatement(viewTableName, PTableType.VIEW, false, true, true));
+   }
+   }
+   }
+   }
+
 private boolean execeededIndexQuota(PTableType tableType, PTable 
parentTable) {
 return PTableType.INDEX == tableType && 
parentTable.getIndexes().size() >= maxIndexesPerTable;
 }
-
-private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
-TableViewFinder result, long clientTimeStamp, int 
clientVersion) throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, 
table, clientVersion, false);
-result.addResult(currResult);
-for (ViewInfo viewInfo : currResult.getViewInfoList()) {
-byte[] viewtenantId = viewInfo.getTenantId();
-byte[] viewSchema = viewInfo.getSchemaName();
-byte[] viewTable = viewInfo.getViewName();
-byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, 
viewSchema, viewTable);
-ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
-PTable view = loadTable(env, tableKey, cacheKey, 
clientTimeStamp, clientTimeStamp, clientVersion);
-if (view == null) {
-logger.warn("Found orphan tenant view row in 
SYSTEM.CATALOG with tenantId:"
-+ Bytes.toString(tenantId) + ", schema:"
-+ Bytes.toString(viewSchema) + ", table:"
-+ Bytes.toString(viewTable));
-continue;
-}
-findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
-}
-}
-
-// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
-// and we have a separate table for links.
-private TableViewFinder findChildViews_deprecated(Region region, 
byte[] tenantId, PTable table, byte[] linkTypeBytes, boolean stopAfterFirst) 
throws IOException {
-byte[] schemaName = table.getSchemaName().getBytes();
-byte[] tableName = table.getTableName().getBytes();
-boolean isMultiTenant = table.isMultiTenant();
-Scan scan = new Scan();
-// If the table is multi-tenant, we need to check across all 
tenant_ids,
-// so we can't constrain the row key. Otherwise, any views would 
have
-// the same tenantId.
-  

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201776174
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1488,20 +1904,19 @@ public void createTable(RpcController controller, 
CreateTableRequest request,
 if (tableType == PTableType.VIEW) {
 byte[][] parentSchemaTableNames = new byte[3][];
 byte[][] parentPhysicalSchemaTableNames = new byte[3][];
-/*
- * For a view, we lock the base physical table row. For a 
mapped view, there is 
- * no link present to the physical table. So the 
viewPhysicalTableRow is null
- * in that case.
- */
+   /*
+* For a mapped view, there is no link present 
to the physical table. So the
+* viewPhysicalTableRow is null in that case.
+*/
--- End diff --

Fixed.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201776136
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1457,28 +1791,110 @@ private static void getSchemaTableNames(Mutation 
row, byte[][] schemaTableNames)
 schemaTableNames[2] = tName;
 }
 }
-
+
 @Override
 public void createTable(RpcController controller, CreateTableRequest 
request,
 RpcCallback done) {
 MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
 byte[][] rowKeyMetaData = new byte[3][];
 byte[] schemaName = null;
 byte[] tableName = null;
+String fullTableName = null;
 try {
 int clientVersion = request.getClientVersion();
 List tableMetadata = 
ProtobufUtil.getMutations(request);
 MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, 
rowKeyMetaData);
 byte[] tenantIdBytes = 
rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 schemaName = 
rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 tableName = 
rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+// TODO before creating a table we need to see if the table 
was previously created and then dropped
+// and clean up any parent->child links or child views
 boolean isNamespaceMapped = 
MetaDataUtil.isNameSpaceMapped(tableMetadata, GenericKeyValueBuilder.INSTANCE,
 new ImmutableBytesWritable());
 final IndexType indexType = 
MetaDataUtil.getIndexType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
 new ImmutableBytesWritable());
+byte[] parentTenantId = null;
 byte[] parentSchemaName = null;
 byte[] parentTableName = null;
 PTableType tableType = 
MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new 
ImmutableBytesWritable());
+ViewType viewType = MetaDataUtil.getViewType(tableMetadata, 
GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
+
+// Load table to see if it already exists
+byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, 
schemaName, tableName);
+ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
+long clientTimeStamp = 
MetaDataUtil.getClientTimeStamp(tableMetadata);
+PTable table = null;
+   try {
+   // Get as of latest timestamp so we can detect 
if we have a newer table that already
+   // exists without making an additional query
+   table = loadTable(env, tableKey, cacheKey, 
clientTimeStamp, HConstants.LATEST_TIMESTAMP,
+   clientVersion);
+   } catch (ParentTableNotFoundException e) {
+   dropChildMetadata(e.getParentSchemaName(), 
e.getParentTableName(), e.getParentTenantId());
+   }
+if (table != null) {
+if (table.getTimeStamp() < clientTimeStamp) {
+// If the table is older than the client time stamp 
and it's deleted,
+// continue
+if (!isTableDeleted(table)) {
+
builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
+
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+builder.setTable(PTableImpl.toProto(table));
+done.run(builder.build());
+return;
+}
+} else {
+
builder.setReturnCode(MetaDataProtos.MutationCode.NEWER_TABLE_FOUND);
+
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+builder.setTable(PTableImpl.toProto(table));
+done.run(builder.build());
+return;
+}
+}
+
+   // check if the table was dropped, but had child views 
that were have not yet
+   // been cleaned up by compaction
+   if 
(!Bytes.toString(schemaName).equals(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+   dropChildMetadata(schemaName, tableName, 
tenantIdBytes);
+   }
--- End diff --

Fixed.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201776009
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -586,48 +590,359 @@ public void getTable(RpcController controller, 
GetTableRequest request,
 
builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
 }
 }
-
-if (table.getTimeStamp() != tableTimeStamp) {
+// the PTable of views and indexes on views might get updated 
because a column is added to one of
+// their parents (this won't change the timestamp)
+if (table.getType()!=PTableType.TABLE || table.getTimeStamp() 
!= tableTimeStamp) {
 builder.setTable(PTableImpl.toProto(table));
 }
 done.run(builder.build());
-return;
 } catch (Throwable t) {
 logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
 }
 
+/**
+ * Used to add the columns present the ancestor hierarchy to the 
PTable of the given view or
+ * view index
+ * @param table PTable of the view or view index
+ * @param skipAddingIndexes if true the returned PTable won't include 
indexes
+ * @param skipAddingParentColumns if true the returned PTable won't 
include columns derived from
+ *ancestor tables
+ * @param lockedAncestorTable ancestor table table that is being 
mutated (as we won't be able to
+ *resolve this table as its locked)
+ */
+private Pair 
combineColumns(PTable table, long timestamp,
+int clientVersion, boolean skipAddingIndexes, boolean 
skipAddingParentColumns,
+PTable lockedAncestorTable) throws SQLException, IOException {
+boolean hasIndexId = table.getViewIndexId() != null;
+if (table.getType() != PTableType.VIEW && !hasIndexId) {
+return new Pair(table,
+MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
+}
+if (!skipAddingParentColumns) {
+table =
+addDerivedColumnsFromAncestors(table, timestamp, 
clientVersion,
+lockedAncestorTable);
+if (table==null) {
+return new Pair(table,
+MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+}
+// we need to resolve the indexes of views (to get ensure they 
also have all the columns
+// derived from their ancestors) 
+if (!skipAddingIndexes && !table.getIndexes().isEmpty()) {
+List indexes = 
Lists.newArrayListWithExpectedSize(table.getIndexes().size());
+for (PTable index : table.getIndexes()) {
+byte[] tenantIdBytes =
+index.getTenantId() == null ? 
ByteUtil.EMPTY_BYTE_ARRAY
+: index.getTenantId().getBytes();
+PTable latestIndex =
+doGetTable(tenantIdBytes, 
index.getSchemaName().getBytes(),
+index.getTableName().getBytes(), 
timestamp, null, clientVersion, true,
+false, lockedAncestorTable);
+if (latestIndex == null) {
+throw new TableNotFoundException(
+"Could not find index table while 
combining columns "
++ index.getTableName().getString() 
+ " with tenant id "
++ index.getTenantId());
+}
+indexes.add(latestIndex);
+}
+table = PTableImpl.makePTable(table, table.getTimeStamp(), 
indexes);
+}
+}
+
+MetaDataProtos.MutationCode mutationCode =
+table != null ? 
MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS
+: MetaDataProtos.MutationCode.TABLE_NOT_FOUND;
+return new Pair(table, 
mutationCode);
+}
+
+
+private PTable addDerivedColumnsFromAncestors(PTable table, long 
timestamp,
+int clientVersion, PTable lockedAncestorTable) throws 
IOException, SQLException, TableNotFoundException {
+// combine columns for view and view indexes
+byte[] tenantId =
+table.getTenantId() != null ? 
table.getTenantId().getBytes()
  

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201776046
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -586,48 +590,359 @@ public void getTable(RpcController controller, 
GetTableRequest request,
 
builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
 }
 }
-
-if (table.getTimeStamp() != tableTimeStamp) {
+// the PTable of views and indexes on views might get updated 
because a column is added to one of
+// their parents (this won't change the timestamp)
+if (table.getType()!=PTableType.TABLE || table.getTimeStamp() 
!= tableTimeStamp) {
 builder.setTable(PTableImpl.toProto(table));
 }
 done.run(builder.build());
-return;
 } catch (Throwable t) {
 logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
 }
 
+/**
+ * Used to add the columns present the ancestor hierarchy to the 
PTable of the given view or
+ * view index
+ * @param table PTable of the view or view index
+ * @param skipAddingIndexes if true the returned PTable won't include 
indexes
+ * @param skipAddingParentColumns if true the returned PTable won't 
include columns derived from
+ *ancestor tables
+ * @param lockedAncestorTable ancestor table table that is being 
mutated (as we won't be able to
+ *resolve this table as its locked)
+ */
+private Pair 
combineColumns(PTable table, long timestamp,
+int clientVersion, boolean skipAddingIndexes, boolean 
skipAddingParentColumns,
+PTable lockedAncestorTable) throws SQLException, IOException {
+boolean hasIndexId = table.getViewIndexId() != null;
+if (table.getType() != PTableType.VIEW && !hasIndexId) {
+return new Pair(table,
+MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
+}
+if (!skipAddingParentColumns) {
+table =
+addDerivedColumnsFromAncestors(table, timestamp, 
clientVersion,
+lockedAncestorTable);
+if (table==null) {
+return new Pair(table,
+MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+}
+// we need to resolve the indexes of views (to get ensure they 
also have all the columns
+// derived from their ancestors) 
+if (!skipAddingIndexes && !table.getIndexes().isEmpty()) {
+List indexes = 
Lists.newArrayListWithExpectedSize(table.getIndexes().size());
+for (PTable index : table.getIndexes()) {
+byte[] tenantIdBytes =
+index.getTenantId() == null ? 
ByteUtil.EMPTY_BYTE_ARRAY
+: index.getTenantId().getBytes();
+PTable latestIndex =
+doGetTable(tenantIdBytes, 
index.getSchemaName().getBytes(),
+index.getTableName().getBytes(), 
timestamp, null, clientVersion, true,
+false, lockedAncestorTable);
+if (latestIndex == null) {
+throw new TableNotFoundException(
+"Could not find index table while 
combining columns "
++ index.getTableName().getString() 
+ " with tenant id "
++ index.getTenantId());
+}
+indexes.add(latestIndex);
+}
+table = PTableImpl.makePTable(table, table.getTimeStamp(), 
indexes);
+}
+}
+
+MetaDataProtos.MutationCode mutationCode =
+table != null ? 
MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS
+: MetaDataProtos.MutationCode.TABLE_NOT_FOUND;
+return new Pair(table, 
mutationCode);
+}
+
+
+private PTable addDerivedColumnsFromAncestors(PTable table, long 
timestamp,
+int clientVersion, PTable lockedAncestorTable) throws 
IOException, SQLException, TableNotFoundException {
+// combine columns for view and view indexes
+byte[] tenantId =
+table.getTenantId() != null ? 
table.getTenantId().getBytes()
  

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201744675
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -2573,307 +2897,139 @@ else if (pkCount <= COLUMN_NAME_INDEX
 return new 
MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, 
EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
 }
 
-ColumnOrdinalPositionUpdateList ordinalPositionList = new 
ColumnOrdinalPositionUpdateList();
+//add the new columns to the child view
 List viewPkCols = new 
ArrayList<>(view.getPKColumns());
 boolean addingExistingPkCol = false;
-int numCols = view.getColumns().size();
-// add the new columns to the child view
-for (PutWithOrdinalPosition p : columnPutsForBaseTable) {
-Put baseTableColumnPut = p.put;
+for (Put columnToBeAdded : columnPutsForBaseTable) {
 PColumn existingViewColumn = null;
 byte[][] rkmd = new byte[5][];
-getVarChars(baseTableColumnPut.getRow(), rkmd);
+getVarChars(columnToBeAdded.getRow(), rkmd);
 String columnName = 
Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
-String columnFamily = rkmd[FAMILY_NAME_INDEX] == null ? 
null : Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
+String columnFamily =
+rkmd[FAMILY_NAME_INDEX] == null ? null
+: Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
 try {
-existingViewColumn = columnFamily == null ? 
view.getColumnForColumnName(columnName) : view.getColumnFamily(
-
columnFamily).getPColumnForColumnName(columnName);
+existingViewColumn =
+columnFamily == null ? 
view.getColumnForColumnName(columnName)
+: view.getColumnFamily(columnFamily)
+
.getPColumnForColumnName(columnName);
 } catch (ColumnFamilyNotFoundException e) {
-// ignore since it means that the column family is not 
present for the column to be added.
+// ignore since it means that the column family is not 
present for the column to
+// be added.
 } catch (ColumnNotFoundException e) {
 // ignore since it means the column is not present in 
the view
 }
-
-boolean isPkCol = columnFamily == null;
-byte[] columnKey = getColumnKey(viewKey, columnName, 
columnFamily);
+
+boolean isColumnToBeAddPkCol = columnFamily == null;
 if (existingViewColumn != null) {
-MetaDataMutationResult result = 
validateColumnForAddToBaseTable(existingViewColumn, baseTableColumnPut, 
basePhysicalTable, isPkCol, view);
-if (result != null) {
-return result;
+if 
(EncodedColumnsUtil.usesEncodedColumnNames(basePhysicalTable)
+&& !SchemaUtil.isPKColumn(existingViewColumn)) 
{
--- End diff --

The race condition with adding the same column to the base table and a 
child view will be covered in PHOENIX-4799.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201741221
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -2147,46 +2566,29 @@ private MetaDataMutationResult doDropTable(byte[] 
key, byte[] tenantId, byte[] s
 }
 
 if (tableType == PTableType.TABLE || tableType == 
PTableType.SYSTEM) {
-// Handle any child views that exist
-TableViewFinder tableViewFinderResult = 
findChildViews(region, tenantId, table, clientVersion, !isCascade);
-if (tableViewFinderResult.hasViews()) {
-if (isCascade) {
-if 
(tableViewFinderResult.allViewsInMultipleRegions()) {
-// We don't yet support deleting a table with 
views where SYSTEM.CATALOG has split and the
-// view metadata spans multiple regions
-return new 
MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-
EnvironmentEdgeManager.currentTimeMillis(), null);
-} else if 
(tableViewFinderResult.allViewsInSingleRegion()) {
-// Recursively delete views - safe as all the 
views as all in the same region
-for (ViewInfo viewInfo : 
tableViewFinderResult.getViewInfoList()) {
-byte[] viewTenantId = 
viewInfo.getTenantId();
-byte[] viewSchemaName = 
viewInfo.getSchemaName();
-byte[] viewName = viewInfo.getViewName();
-byte[] viewKey = 
SchemaUtil.getTableKey(viewTenantId, viewSchemaName, viewName);
-Delete delete = new Delete(viewKey, 
clientTimeStamp);
-rowsToDelete.add(delete);
-acquireLock(region, viewKey, locks);
-MetaDataMutationResult result = 
doDropTable(viewKey, viewTenantId, viewSchemaName,
-viewName, null, PTableType.VIEW, 
rowsToDelete, invalidateList, locks,
-tableNamesToDelete, 
sharedTablesToDelete, false, clientVersion);
-if (result.getMutationCode() != 
MutationCode.TABLE_ALREADY_EXISTS) { return result; }
-}
-}
-} else {
+// check to see if the table has any child views
+try (Table hTable =
+env.getTable(SchemaUtil.getPhysicalTableName(
+
PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+env.getConfiguration( {
+boolean hasChildViews =
+ViewFinder.hasChildViews(hTable, tenantId, 
schemaName, tableName,
+clientTimeStamp);
+if (hasChildViews && !isCascade) {
--- End diff --

I think we can handle the race condition in a similar way to how we handle 
conflicting columns using checkAndMutate. I have updated PHOENIX-4799 to 
include this case.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-11 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r201739480
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1809,180 +2256,97 @@ public void createTable(RpcController controller, 
CreateTableRequest request,
 } catch (Throwable t) {
 logger.error("createTable failed", t);
 ProtobufUtil.setControllerException(controller,
-
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
+ServerUtil.createIOException(fullTableName, t));
 }
 }
 
+   private void dropChildMetadata(byte[] schemaName, byte[] tableName, 
byte[] tenantIdBytes)
+   throws IOException, SQLException, 
ClassNotFoundException {
+   TableViewFinderResult childViewsResult = new 
TableViewFinderResult();
+   findAllChildViews(tenantIdBytes, schemaName, tableName, 
childViewsResult);
+   if (childViewsResult.hasViews()) {
+   for (TableInfo viewInfo : 
childViewsResult.getResults()) {
+   byte[] viewTenantId = viewInfo.getTenantId();
+   byte[] viewSchemaName = 
viewInfo.getSchemaName();
+   byte[] viewName = viewInfo.getTableName();
+   Properties props = new Properties();
+   if (viewTenantId != null && viewTenantId.length 
!= 0)
+   
props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, 
Bytes.toString(viewTenantId));
+   try (PhoenixConnection connection = 
QueryUtil.getConnectionOnServer(env.getConfiguration())
+   
.unwrap(PhoenixConnection.class)) {
+   MetaDataClient client = new 
MetaDataClient(connection);
+   org.apache.phoenix.parse.TableName 
viewTableName = org.apache.phoenix.parse.TableName
+   
.create(Bytes.toString(viewSchemaName), Bytes.toString(viewName));
+   client.dropTable(
+   new 
DropTableStatement(viewTableName, PTableType.VIEW, false, true, true));
+   }
+   }
+   }
+   }
+
 private boolean execeededIndexQuota(PTableType tableType, PTable 
parentTable) {
 return PTableType.INDEX == tableType && 
parentTable.getIndexes().size() >= maxIndexesPerTable;
 }
-
-private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
-TableViewFinder result, long clientTimeStamp, int 
clientVersion) throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, 
table, clientVersion, false);
-result.addResult(currResult);
-for (ViewInfo viewInfo : currResult.getViewInfoList()) {
-byte[] viewtenantId = viewInfo.getTenantId();
-byte[] viewSchema = viewInfo.getSchemaName();
-byte[] viewTable = viewInfo.getViewName();
-byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, 
viewSchema, viewTable);
-ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
-PTable view = loadTable(env, tableKey, cacheKey, 
clientTimeStamp, clientTimeStamp, clientVersion);
-if (view == null) {
-logger.warn("Found orphan tenant view row in 
SYSTEM.CATALOG with tenantId:"
-+ Bytes.toString(tenantId) + ", schema:"
-+ Bytes.toString(viewSchema) + ", table:"
-+ Bytes.toString(viewTable));
-continue;
-}
-findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
-}
-}
-
-// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
-// and we have a separate table for links.
-private TableViewFinder findChildViews_deprecated(Region region, 
byte[] tenantId, PTable table, byte[] linkTypeBytes, boolean stopAfterFirst) 
throws IOException {
-byte[] schemaName = table.getSchemaName().getBytes();
-byte[] tableName = table.getTableName().getBytes();
-boolean isMultiTenant = table.isMultiTenant();
-Scan scan = new Scan();
-// If the table is multi-tenant, we need to check across all 
tenant_ids,
-// so we can't constrain the row key. Otherwise, any views would 
have
-// the same tenantId.
-  

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200208444
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java ---
@@ -1893,26 +1981,45 @@ public static void upgradeTable(PhoenixConnection 
conn, String srcTable) throws
 conn.commit();
 }
 conn.getQueryServices().clearTableFromCache(
-conn.getTenantId() == null ? 
ByteUtil.EMPTY_BYTE_ARRAY : conn.getTenantId().getBytes(),
+tenantIdBytes,
 index.getSchemaName().getBytes(), 
index.getTableName().getBytes(),
 PhoenixRuntime.getCurrentScn(readOnlyProps));
 }
 updateIndexesSequenceIfPresent(conn, table);
 conn.commit();
-
 } else {
 throw new RuntimeException("Error: problem occured during 
upgrade. Table is not upgraded successfully");
 }
 if (table.getType() == PTableType.VIEW) {
 logger.info(String.format("Updating link information for 
view '%s' ..", table.getTableName()));
 updateLink(conn, oldPhysicalName, 
newPhysicalTablename,table.getSchemaName(),table.getTableName());
 conn.commit();
-
+
+// if the view is a first level child, then we need to 
create the PARENT_TABLE link
+// that was overwritten by the PHYSICAL_TABLE link 
--- End diff --

Ah, good. So we'll be consistent with the parent link now, right?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200208319
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1779,13 +2174,65 @@ public void createTable(RpcController controller, 
CreateTableRequest request,
 }
 }
 
+// The mutations to create a table are written in the 
following order:
+// 1. Write the child link as if the next two steps fail we
+// ignore missing children while processing a parent
+// 2. Update the encoded column qualifier for the parent 
table if its on a
+// different region server (for tables that use column 
qualifier encoding)
+// if the next step fails we end up wasting a few col 
qualifiers
+// 3. Finally write the mutations to create the table
+
+// From 4.15 the parent->child links are stored in a 
separate table SYSTEM.CHILD_LINK
+List childLinkMutations = 
MetaDataUtil.removeChildLinks(tableMetadata);
--- End diff --

TODO to remove this code in 4.16. 


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200209126
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -372,6 +378,31 @@ public void testViewAndTableAndDrop() throws Exception 
{
 // drop table cascade should succeed
 conn.createStatement().execute("DROP TABLE " + fullTableName + " 
CASCADE");
 
+validateViewDoesNotExist(conn, fullViewName1);
+validateViewDoesNotExist(conn, fullViewName2);
+
+}
+
+@Test
+public void testRecreateDroppedTableWithChildViews() throws Exception {
--- End diff --

These new tests are good. These are testing that the left over metadata 
doesn't impact the re-creation of a table since we don't make the RPC to delete 
views when a base table is dropped, right? Do you think there'd be any issues 
if part of the rows for a view were there (i.e. say that the create view 
failed, but some of the rows were written)? Might be good to have a test like 
this - you could set it up by using HBase APIs to manually delete some rows of 
a view.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200206862
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1809,180 +2256,97 @@ public void createTable(RpcController controller, 
CreateTableRequest request,
 } catch (Throwable t) {
 logger.error("createTable failed", t);
 ProtobufUtil.setControllerException(controller,
-
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
+ServerUtil.createIOException(fullTableName, t));
 }
 }
 
+   private void dropChildMetadata(byte[] schemaName, byte[] tableName, 
byte[] tenantIdBytes)
+   throws IOException, SQLException, 
ClassNotFoundException {
+   TableViewFinderResult childViewsResult = new 
TableViewFinderResult();
+   findAllChildViews(tenantIdBytes, schemaName, tableName, 
childViewsResult);
+   if (childViewsResult.hasViews()) {
+   for (TableInfo viewInfo : 
childViewsResult.getResults()) {
+   byte[] viewTenantId = viewInfo.getTenantId();
+   byte[] viewSchemaName = 
viewInfo.getSchemaName();
+   byte[] viewName = viewInfo.getTableName();
+   Properties props = new Properties();
+   if (viewTenantId != null && viewTenantId.length 
!= 0)
+   
props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, 
Bytes.toString(viewTenantId));
+   try (PhoenixConnection connection = 
QueryUtil.getConnectionOnServer(env.getConfiguration())
+   
.unwrap(PhoenixConnection.class)) {
+   MetaDataClient client = new 
MetaDataClient(connection);
+   org.apache.phoenix.parse.TableName 
viewTableName = org.apache.phoenix.parse.TableName
+   
.create(Bytes.toString(viewSchemaName), Bytes.toString(viewName));
+   client.dropTable(
+   new 
DropTableStatement(viewTableName, PTableType.VIEW, false, true, true));
+   }
+   }
+   }
+   }
+
 private boolean execeededIndexQuota(PTableType tableType, PTable 
parentTable) {
 return PTableType.INDEX == tableType && 
parentTable.getIndexes().size() >= maxIndexesPerTable;
 }
-
-private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
-TableViewFinder result, long clientTimeStamp, int 
clientVersion) throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, 
table, clientVersion, false);
-result.addResult(currResult);
-for (ViewInfo viewInfo : currResult.getViewInfoList()) {
-byte[] viewtenantId = viewInfo.getTenantId();
-byte[] viewSchema = viewInfo.getSchemaName();
-byte[] viewTable = viewInfo.getViewName();
-byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, 
viewSchema, viewTable);
-ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
-PTable view = loadTable(env, tableKey, cacheKey, 
clientTimeStamp, clientTimeStamp, clientVersion);
-if (view == null) {
-logger.warn("Found orphan tenant view row in 
SYSTEM.CATALOG with tenantId:"
-+ Bytes.toString(tenantId) + ", schema:"
-+ Bytes.toString(viewSchema) + ", table:"
-+ Bytes.toString(viewTable));
-continue;
-}
-findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
-}
-}
-
-// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
-// and we have a separate table for links.
-private TableViewFinder findChildViews_deprecated(Region region, 
byte[] tenantId, PTable table, byte[] linkTypeBytes, boolean stopAfterFirst) 
throws IOException {
-byte[] schemaName = table.getSchemaName().getBytes();
-byte[] tableName = table.getTableName().getBytes();
-boolean isMultiTenant = table.isMultiTenant();
-Scan scan = new Scan();
-// If the table is multi-tenant, we need to check across all 
tenant_ids,
-// so we can't constrain the row key. Otherwise, any views would 
have
-// the same tenantId.
   

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200208028
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -447,7 +447,7 @@
 static {
 Collections.sort(FUNCTION_KV_COLUMNS, KeyValue.COMPARATOR);
 }
-
+
--- End diff --

Might be good to include a class level comment that explains the overall 
approach at a high level.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200208160
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java ---
@@ -231,36 +231,26 @@ public Object getPTableValue(PTable table) {
 private final SQLExceptionCode mutatingImmutablePropException;
 private final boolean isValidOnView;
 private final boolean isMutableOnView;
-private final boolean propagateToViews;
 
 private TableProperty(String propertyName, boolean isMutable, boolean 
isValidOnView, boolean isMutableOnView) {
-this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, true);
-}
-
-private TableProperty(String propertyName, boolean isMutable, boolean 
isValidOnView, boolean isMutableOnView, boolean propagateToViews) {
-this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, 
propagateToViews);
+this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView);
 }
 
 private TableProperty(String propertyName, SQLExceptionCode 
colFamilySpecifiedException, boolean isMutable, boolean isValidOnView, boolean 
isMutableOnView) {
-this(propertyName, colFamilySpecifiedException, isMutable, 
CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, true);
+this(propertyName, colFamilySpecifiedException, isMutable, 
CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView);
 }
 
 private TableProperty(String propertyName, boolean isMutable, boolean 
isValidOnView, boolean isMutableOnView, SQLExceptionCode isMutatingException) {
-this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, isMutatingException, isValidOnView, isMutableOnView, true);
+this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, 
isMutable, isMutatingException, isValidOnView, isMutableOnView);
 }
 
 private TableProperty(String propertyName, SQLExceptionCode 
colFamSpecifiedException, boolean isMutable, SQLExceptionCode 
mutatingException, boolean isValidOnView, boolean isMutableOnView) {
-this(propertyName, colFamSpecifiedException, isMutable, 
mutatingException, isValidOnView, isMutableOnView, true);
-}
-
-private TableProperty(String propertyName, SQLExceptionCode 
colFamSpecifiedException, boolean isMutable, SQLExceptionCode 
mutatingException, boolean isValidOnView, boolean isMutableOnView, boolean 
propagateToViews) {
--- End diff --

How did you end up dealing with table property conflicts between parent and 
children? Is there follow up work required? Can we use the timestamp of the 
Cell storing the property to differentiate similar to the logic for columns? 
It's fine to do this work in a follow up JIRA.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200207759
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---
@@ -1957,6 +1968,17 @@ private PTable 
createTableInternal(CreateTableStatement statement, byte[][] spli
 linkStatement.setLong(6, parent.getSequenceNumber());
 linkStatement.setString(7, 
PTableType.INDEX.getSerializedValue());
 linkStatement.execute();
+
+// Add row linking index table to parent table for indexes 
on views
+if (parent.getType() == PTableType.VIEW) {
+   linkStatement = 
connection.prepareStatement(CREATE_VIEW_INDEX_PARENT_LINK);
+   linkStatement.setString(1, tenantIdStr);
+   linkStatement.setString(2, schemaName);
+   linkStatement.setString(3, tableName);
+   linkStatement.setString(4, 
parent.getName().getString());
+   linkStatement.setByte(5, 
LinkType.VIEW_INDEX_PARENT_TABLE.getSerializedValue());
+   linkStatement.execute();
+}
--- End diff --

We need to update MetaDataClient.createTableInternal() to not include the 
columns from the parent table in 4.15 so that we can remove the code in 
MetaDataEndPointImpl that filters the columns. It's fine to do this in a follow 
up JIRA, but we should remember to do it.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200207594
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -3642,30 +3596,62 @@ private void dropIndexes(PTable table, Region 
region, List in
 boolean isCoveredColumn = 
indexMaintainer.getCoveredColumns().contains(colDropRef);
 // If index requires this column for its pk, then drop it
 if (isColumnIndexed) {
-// Since we're dropping the index, lock it to ensure
-// that a change in index state doesn't
-// occur while we're dropping it.
-acquireLock(region, indexKey, locks);
 // Drop the index table. The doDropTable will expand
 // this to all of the table rows and invalidate the
 // index table
-additionalTableMetaData.add(new Delete(indexKey, 
clientTimeStamp));
+Delete delete = new Delete(indexKey, clientTimeStamp);
 byte[] linkKey =
 MetaDataUtil.getParentLinkKey(tenantId, 
schemaName, tableName, index
 .getTableName().getBytes());
-// Drop the link between the data table and the
+// Drop the link between the parent table and the
 // index table
-additionalTableMetaData.add(new Delete(linkKey, 
clientTimeStamp));
-doDropTable(indexKey, tenantId, 
index.getSchemaName().getBytes(), index
-.getTableName().getBytes(), tableName, 
index.getType(),
-additionalTableMetaData, invalidateList, locks, 
tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
-invalidateList.add(new ImmutableBytesPtr(indexKey));
+Delete linkDelete = new Delete(linkKey, clientTimeStamp);
+List tableMetaData = 
Lists.newArrayListWithExpectedSize(2);
+Delete tableDelete = delete;
+tableMetaData.add(tableDelete);
+tableMetaData.add(linkDelete);
+// if the index is not present on the current region make 
an rpc to drop it
--- End diff --

Is this ever the case since the index should be in the same schema as it's 
table? Or is there a corner case with indexes on views?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200207388
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -2573,307 +2897,139 @@ else if (pkCount <= COLUMN_NAME_INDEX
 return new 
MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, 
EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
 }
 
-ColumnOrdinalPositionUpdateList ordinalPositionList = new 
ColumnOrdinalPositionUpdateList();
+//add the new columns to the child view
 List viewPkCols = new 
ArrayList<>(view.getPKColumns());
 boolean addingExistingPkCol = false;
-int numCols = view.getColumns().size();
-// add the new columns to the child view
-for (PutWithOrdinalPosition p : columnPutsForBaseTable) {
-Put baseTableColumnPut = p.put;
+for (Put columnToBeAdded : columnPutsForBaseTable) {
 PColumn existingViewColumn = null;
 byte[][] rkmd = new byte[5][];
-getVarChars(baseTableColumnPut.getRow(), rkmd);
+getVarChars(columnToBeAdded.getRow(), rkmd);
 String columnName = 
Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
-String columnFamily = rkmd[FAMILY_NAME_INDEX] == null ? 
null : Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
+String columnFamily =
+rkmd[FAMILY_NAME_INDEX] == null ? null
+: Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
 try {
-existingViewColumn = columnFamily == null ? 
view.getColumnForColumnName(columnName) : view.getColumnFamily(
-
columnFamily).getPColumnForColumnName(columnName);
+existingViewColumn =
+columnFamily == null ? 
view.getColumnForColumnName(columnName)
+: view.getColumnFamily(columnFamily)
+
.getPColumnForColumnName(columnName);
 } catch (ColumnFamilyNotFoundException e) {
-// ignore since it means that the column family is not 
present for the column to be added.
+// ignore since it means that the column family is not 
present for the column to
+// be added.
 } catch (ColumnNotFoundException e) {
 // ignore since it means the column is not present in 
the view
 }
-
-boolean isPkCol = columnFamily == null;
-byte[] columnKey = getColumnKey(viewKey, columnName, 
columnFamily);
+
+boolean isColumnToBeAddPkCol = columnFamily == null;
 if (existingViewColumn != null) {
-MetaDataMutationResult result = 
validateColumnForAddToBaseTable(existingViewColumn, baseTableColumnPut, 
basePhysicalTable, isPkCol, view);
-if (result != null) {
-return result;
+if 
(EncodedColumnsUtil.usesEncodedColumnNames(basePhysicalTable)
+&& !SchemaUtil.isPKColumn(existingViewColumn)) 
{
--- End diff --

Is there a race condition with this check and would the be covered by one 
of the future JIRAs you mentioned?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200205809
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -586,48 +590,359 @@ public void getTable(RpcController controller, 
GetTableRequest request,
 
builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
 }
 }
-
-if (table.getTimeStamp() != tableTimeStamp) {
+// the PTable of views and indexes on views might get updated 
because a column is added to one of
+// their parents (this won't change the timestamp)
+if (table.getType()!=PTableType.TABLE || table.getTimeStamp() 
!= tableTimeStamp) {
 builder.setTable(PTableImpl.toProto(table));
 }
 done.run(builder.build());
-return;
 } catch (Throwable t) {
 logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
 }
 
+/**
+ * Used to add the columns present the ancestor hierarchy to the 
PTable of the given view or
+ * view index
+ * @param table PTable of the view or view index
+ * @param skipAddingIndexes if true the returned PTable won't include 
indexes
+ * @param skipAddingParentColumns if true the returned PTable won't 
include columns derived from
+ *ancestor tables
+ * @param lockedAncestorTable ancestor table table that is being 
mutated (as we won't be able to
+ *resolve this table as its locked)
+ */
+private Pair 
combineColumns(PTable table, long timestamp,
+int clientVersion, boolean skipAddingIndexes, boolean 
skipAddingParentColumns,
+PTable lockedAncestorTable) throws SQLException, IOException {
+boolean hasIndexId = table.getViewIndexId() != null;
+if (table.getType() != PTableType.VIEW && !hasIndexId) {
+return new Pair(table,
+MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
+}
+if (!skipAddingParentColumns) {
+table =
+addDerivedColumnsFromAncestors(table, timestamp, 
clientVersion,
+lockedAncestorTable);
+if (table==null) {
+return new Pair(table,
+MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+}
+// we need to resolve the indexes of views (to get ensure they 
also have all the columns
+// derived from their ancestors) 
+if (!skipAddingIndexes && !table.getIndexes().isEmpty()) {
+List indexes = 
Lists.newArrayListWithExpectedSize(table.getIndexes().size());
+for (PTable index : table.getIndexes()) {
+byte[] tenantIdBytes =
+index.getTenantId() == null ? 
ByteUtil.EMPTY_BYTE_ARRAY
+: index.getTenantId().getBytes();
+PTable latestIndex =
+doGetTable(tenantIdBytes, 
index.getSchemaName().getBytes(),
+index.getTableName().getBytes(), 
timestamp, null, clientVersion, true,
+false, lockedAncestorTable);
+if (latestIndex == null) {
+throw new TableNotFoundException(
+"Could not find index table while 
combining columns "
++ index.getTableName().getString() 
+ " with tenant id "
++ index.getTenantId());
+}
+indexes.add(latestIndex);
+}
+table = PTableImpl.makePTable(table, table.getTimeStamp(), 
indexes);
+}
+}
+
+MetaDataProtos.MutationCode mutationCode =
+table != null ? 
MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS
+: MetaDataProtos.MutationCode.TABLE_NOT_FOUND;
+return new Pair(table, 
mutationCode);
+}
+
+
+private PTable addDerivedColumnsFromAncestors(PTable table, long 
timestamp,
+int clientVersion, PTable lockedAncestorTable) throws 
IOException, SQLException, TableNotFoundException {
+// combine columns for view and view indexes
+byte[] tenantId =
+table.getTenantId() != null ? 

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200206109
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -586,48 +590,359 @@ public void getTable(RpcController controller, 
GetTableRequest request,
 
builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
 }
 }
-
-if (table.getTimeStamp() != tableTimeStamp) {
+// the PTable of views and indexes on views might get updated 
because a column is added to one of
+// their parents (this won't change the timestamp)
+if (table.getType()!=PTableType.TABLE || table.getTimeStamp() 
!= tableTimeStamp) {
 builder.setTable(PTableImpl.toProto(table));
 }
 done.run(builder.build());
-return;
 } catch (Throwable t) {
 logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
 }
 
+/**
+ * Used to add the columns present the ancestor hierarchy to the 
PTable of the given view or
+ * view index
+ * @param table PTable of the view or view index
+ * @param skipAddingIndexes if true the returned PTable won't include 
indexes
+ * @param skipAddingParentColumns if true the returned PTable won't 
include columns derived from
+ *ancestor tables
+ * @param lockedAncestorTable ancestor table table that is being 
mutated (as we won't be able to
+ *resolve this table as its locked)
+ */
+private Pair 
combineColumns(PTable table, long timestamp,
+int clientVersion, boolean skipAddingIndexes, boolean 
skipAddingParentColumns,
+PTable lockedAncestorTable) throws SQLException, IOException {
+boolean hasIndexId = table.getViewIndexId() != null;
+if (table.getType() != PTableType.VIEW && !hasIndexId) {
+return new Pair(table,
+MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
+}
+if (!skipAddingParentColumns) {
+table =
+addDerivedColumnsFromAncestors(table, timestamp, 
clientVersion,
+lockedAncestorTable);
+if (table==null) {
+return new Pair(table,
+MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+}
+// we need to resolve the indexes of views (to get ensure they 
also have all the columns
+// derived from their ancestors) 
+if (!skipAddingIndexes && !table.getIndexes().isEmpty()) {
+List indexes = 
Lists.newArrayListWithExpectedSize(table.getIndexes().size());
+for (PTable index : table.getIndexes()) {
+byte[] tenantIdBytes =
+index.getTenantId() == null ? 
ByteUtil.EMPTY_BYTE_ARRAY
+: index.getTenantId().getBytes();
+PTable latestIndex =
+doGetTable(tenantIdBytes, 
index.getSchemaName().getBytes(),
+index.getTableName().getBytes(), 
timestamp, null, clientVersion, true,
+false, lockedAncestorTable);
+if (latestIndex == null) {
+throw new TableNotFoundException(
+"Could not find index table while 
combining columns "
++ index.getTableName().getString() 
+ " with tenant id "
++ index.getTenantId());
+}
+indexes.add(latestIndex);
+}
+table = PTableImpl.makePTable(table, table.getTimeStamp(), 
indexes);
+}
+}
+
+MetaDataProtos.MutationCode mutationCode =
+table != null ? 
MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS
+: MetaDataProtos.MutationCode.TABLE_NOT_FOUND;
+return new Pair(table, 
mutationCode);
+}
+
+
+private PTable addDerivedColumnsFromAncestors(PTable table, long 
timestamp,
+int clientVersion, PTable lockedAncestorTable) throws 
IOException, SQLException, TableNotFoundException {
+// combine columns for view and view indexes
+byte[] tenantId =
+table.getTenantId() != null ? 

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200207100
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -2147,46 +2566,29 @@ private MetaDataMutationResult doDropTable(byte[] 
key, byte[] tenantId, byte[] s
 }
 
 if (tableType == PTableType.TABLE || tableType == 
PTableType.SYSTEM) {
-// Handle any child views that exist
-TableViewFinder tableViewFinderResult = 
findChildViews(region, tenantId, table, clientVersion, !isCascade);
-if (tableViewFinderResult.hasViews()) {
-if (isCascade) {
-if 
(tableViewFinderResult.allViewsInMultipleRegions()) {
-// We don't yet support deleting a table with 
views where SYSTEM.CATALOG has split and the
-// view metadata spans multiple regions
-return new 
MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-
EnvironmentEdgeManager.currentTimeMillis(), null);
-} else if 
(tableViewFinderResult.allViewsInSingleRegion()) {
-// Recursively delete views - safe as all the 
views as all in the same region
-for (ViewInfo viewInfo : 
tableViewFinderResult.getViewInfoList()) {
-byte[] viewTenantId = 
viewInfo.getTenantId();
-byte[] viewSchemaName = 
viewInfo.getSchemaName();
-byte[] viewName = viewInfo.getViewName();
-byte[] viewKey = 
SchemaUtil.getTableKey(viewTenantId, viewSchemaName, viewName);
-Delete delete = new Delete(viewKey, 
clientTimeStamp);
-rowsToDelete.add(delete);
-acquireLock(region, viewKey, locks);
-MetaDataMutationResult result = 
doDropTable(viewKey, viewTenantId, viewSchemaName,
-viewName, null, PTableType.VIEW, 
rowsToDelete, invalidateList, locks,
-tableNamesToDelete, 
sharedTablesToDelete, false, clientVersion);
-if (result.getMutationCode() != 
MutationCode.TABLE_ALREADY_EXISTS) { return result; }
-}
-}
-} else {
+// check to see if the table has any child views
+try (Table hTable =
+env.getTable(SchemaUtil.getPhysicalTableName(
+
PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+env.getConfiguration( {
+boolean hasChildViews =
+ViewFinder.hasChildViews(hTable, tenantId, 
schemaName, tableName,
+clientTimeStamp);
+if (hasChildViews && !isCascade) {
--- End diff --

Isn't there a race condition with this check?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200206293
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1457,28 +1791,110 @@ private static void getSchemaTableNames(Mutation 
row, byte[][] schemaTableNames)
 schemaTableNames[2] = tName;
 }
 }
-
+
 @Override
 public void createTable(RpcController controller, CreateTableRequest 
request,
 RpcCallback done) {
 MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
 byte[][] rowKeyMetaData = new byte[3][];
 byte[] schemaName = null;
 byte[] tableName = null;
+String fullTableName = null;
 try {
 int clientVersion = request.getClientVersion();
 List tableMetadata = 
ProtobufUtil.getMutations(request);
 MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, 
rowKeyMetaData);
 byte[] tenantIdBytes = 
rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 schemaName = 
rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 tableName = 
rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+// TODO before creating a table we need to see if the table 
was previously created and then dropped
+// and clean up any parent->child links or child views
--- End diff --

Remove TODO as isn't this done now?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200206428
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1488,20 +1904,19 @@ public void createTable(RpcController controller, 
CreateTableRequest request,
 if (tableType == PTableType.VIEW) {
 byte[][] parentSchemaTableNames = new byte[3][];
 byte[][] parentPhysicalSchemaTableNames = new byte[3][];
-/*
- * For a view, we lock the base physical table row. For a 
mapped view, there is 
- * no link present to the physical table. So the 
viewPhysicalTableRow is null
- * in that case.
- */
+   /*
+* For a mapped view, there is no link present 
to the physical table. So the
+* viewPhysicalTableRow is null in that case.
+*/
--- End diff --

Fix indentation


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200206792
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1809,180 +2256,97 @@ public void createTable(RpcController controller, 
CreateTableRequest request,
 } catch (Throwable t) {
 logger.error("createTable failed", t);
 ProtobufUtil.setControllerException(controller,
-
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
+ServerUtil.createIOException(fullTableName, t));
 }
 }
 
+   private void dropChildMetadata(byte[] schemaName, byte[] tableName, 
byte[] tenantIdBytes)
+   throws IOException, SQLException, 
ClassNotFoundException {
+   TableViewFinderResult childViewsResult = new 
TableViewFinderResult();
+   findAllChildViews(tenantIdBytes, schemaName, tableName, 
childViewsResult);
+   if (childViewsResult.hasViews()) {
+   for (TableInfo viewInfo : 
childViewsResult.getResults()) {
+   byte[] viewTenantId = viewInfo.getTenantId();
+   byte[] viewSchemaName = 
viewInfo.getSchemaName();
+   byte[] viewName = viewInfo.getTableName();
+   Properties props = new Properties();
+   if (viewTenantId != null && viewTenantId.length 
!= 0)
+   
props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, 
Bytes.toString(viewTenantId));
+   try (PhoenixConnection connection = 
QueryUtil.getConnectionOnServer(env.getConfiguration())
+   
.unwrap(PhoenixConnection.class)) {
+   MetaDataClient client = new 
MetaDataClient(connection);
+   org.apache.phoenix.parse.TableName 
viewTableName = org.apache.phoenix.parse.TableName
+   
.create(Bytes.toString(viewSchemaName), Bytes.toString(viewName));
+   client.dropTable(
+   new 
DropTableStatement(viewTableName, PTableType.VIEW, false, true, true));
+   }
+   }
+   }
+   }
+
 private boolean execeededIndexQuota(PTableType tableType, PTable 
parentTable) {
 return PTableType.INDEX == tableType && 
parentTable.getIndexes().size() >= maxIndexesPerTable;
 }
-
-private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
-TableViewFinder result, long clientTimeStamp, int 
clientVersion) throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, 
table, clientVersion, false);
-result.addResult(currResult);
-for (ViewInfo viewInfo : currResult.getViewInfoList()) {
-byte[] viewtenantId = viewInfo.getTenantId();
-byte[] viewSchema = viewInfo.getSchemaName();
-byte[] viewTable = viewInfo.getViewName();
-byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, 
viewSchema, viewTable);
-ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
-PTable view = loadTable(env, tableKey, cacheKey, 
clientTimeStamp, clientTimeStamp, clientVersion);
-if (view == null) {
-logger.warn("Found orphan tenant view row in 
SYSTEM.CATALOG with tenantId:"
-+ Bytes.toString(tenantId) + ", schema:"
-+ Bytes.toString(viewSchema) + ", table:"
-+ Bytes.toString(viewTable));
-continue;
-}
-findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
-}
-}
-
-// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
-// and we have a separate table for links.
-private TableViewFinder findChildViews_deprecated(Region region, 
byte[] tenantId, PTable table, byte[] linkTypeBytes, boolean stopAfterFirst) 
throws IOException {
-byte[] schemaName = table.getSchemaName().getBytes();
-byte[] tableName = table.getTableName().getBytes();
-boolean isMultiTenant = table.isMultiTenant();
-Scan scan = new Scan();
-// If the table is multi-tenant, we need to check across all 
tenant_ids,
-// so we can't constrain the row key. Otherwise, any views would 
have
-// the same tenantId.
   

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-07-04 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r200206389
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1457,28 +1791,110 @@ private static void getSchemaTableNames(Mutation 
row, byte[][] schemaTableNames)
 schemaTableNames[2] = tName;
 }
 }
-
+
 @Override
 public void createTable(RpcController controller, CreateTableRequest 
request,
 RpcCallback done) {
 MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
 byte[][] rowKeyMetaData = new byte[3][];
 byte[] schemaName = null;
 byte[] tableName = null;
+String fullTableName = null;
 try {
 int clientVersion = request.getClientVersion();
 List tableMetadata = 
ProtobufUtil.getMutations(request);
 MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, 
rowKeyMetaData);
 byte[] tenantIdBytes = 
rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 schemaName = 
rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 tableName = 
rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+// TODO before creating a table we need to see if the table 
was previously created and then dropped
+// and clean up any parent->child links or child views
 boolean isNamespaceMapped = 
MetaDataUtil.isNameSpaceMapped(tableMetadata, GenericKeyValueBuilder.INSTANCE,
 new ImmutableBytesWritable());
 final IndexType indexType = 
MetaDataUtil.getIndexType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
 new ImmutableBytesWritable());
+byte[] parentTenantId = null;
 byte[] parentSchemaName = null;
 byte[] parentTableName = null;
 PTableType tableType = 
MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new 
ImmutableBytesWritable());
+ViewType viewType = MetaDataUtil.getViewType(tableMetadata, 
GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
+
+// Load table to see if it already exists
+byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, 
schemaName, tableName);
+ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
+long clientTimeStamp = 
MetaDataUtil.getClientTimeStamp(tableMetadata);
+PTable table = null;
+   try {
+   // Get as of latest timestamp so we can detect 
if we have a newer table that already
+   // exists without making an additional query
+   table = loadTable(env, tableKey, cacheKey, 
clientTimeStamp, HConstants.LATEST_TIMESTAMP,
+   clientVersion);
+   } catch (ParentTableNotFoundException e) {
+   dropChildMetadata(e.getParentSchemaName(), 
e.getParentTableName(), e.getParentTenantId());
+   }
+if (table != null) {
+if (table.getTimeStamp() < clientTimeStamp) {
+// If the table is older than the client time stamp 
and it's deleted,
+// continue
+if (!isTableDeleted(table)) {
+
builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
+
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+builder.setTable(PTableImpl.toProto(table));
+done.run(builder.build());
+return;
+}
+} else {
+
builder.setReturnCode(MetaDataProtos.MutationCode.NEWER_TABLE_FOUND);
+
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+builder.setTable(PTableImpl.toProto(table));
+done.run(builder.build());
+return;
+}
+}
+
+   // check if the table was dropped, but had child views 
that were have not yet
+   // been cleaned up by compaction
+   if 
(!Bytes.toString(schemaName).equals(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+   dropChildMetadata(schemaName, tableName, 
tenantIdBytes);
+   }
--- End diff --

Minor - indentation issue here.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-29 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r199147322
  
--- Diff: 
phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
 ---
@@ -0,0 +1,299 @@
+package org.apache.phoenix.coprocessor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * 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.
+ */
+public class MetaDataEndpointImplTest extends ParallelStatsDisabledIT {
--- End diff --

@JamesRTaylor Can you please review? I modified the PR to drop child 
metadata when we try to reuse a table name of a table that was dropped. It also 
handles reusing a view name / view index name. 
I also added a config that determines whether or not SYSTEM.CATALOG can 
split. I merged the latest changes from master as well. 


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-25 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r197829637
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
 ---
@@ -35,20 +35,18 @@
  * during cluster upgrades. However, tenant-owned data such as 
tenant-owned views need to
  * be copied. This WALEntryFilter will only allow tenant-owned rows in 
SYSTEM.CATALOG to
  * be replicated. Data from all other tables is automatically passed. It 
will also copy
- * child links in SYSTEM.CATALOG that are globally-owned but point to 
tenant-owned views.
+ * child links in SYSTEM.CHILD_LINK that are globally-owned but point to 
tenant-owned views.
  *
  */
 public class SystemCatalogWALEntryFilter implements WALEntryFilter {
 
-  private static byte[] CHILD_TABLE_BYTES =
-  new byte[]{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
   @Override
   public WAL.Entry filter(WAL.Entry entry) {
 
-//if the WAL.Entry's table isn't System.Catalog, it auto-passes this 
filter
+//if the WAL.Entry's table isn't System.Catalog or System.Child_Link, 
it auto-passes this filter
 //TODO: when Phoenix drops support for pre-1.3 versions of HBase, redo 
as a WALCellFilter
-if (!SchemaUtil.isMetaTable(entry.getKey().getTablename().getName())){
+byte[] tableName = entry.getKey().getTablename().getName();
+   if (!SchemaUtil.isMetaTable(tableName) && 
!SchemaUtil.isChildLinkTable(tableName)){
--- End diff --

SYSTEM.CHILD_LINK contains the parent->child linking rows and cells we use 
to detect race conditions (eg a column of conflicting type being added at the 
same time to a parent and child). 
The latter cells are written with a short TTL. 
I think we can use HBase replication for SYSTEM.CHILD_LINK. All the tenant 
specific view metadata rows in SYSTEM.CATALOG start with tenant id. 
I will modify this filter to how it was before PHOENIX-4229. 
@gjacoby126  Thanks for the suggestion.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-25 Thread gjacoby126
Github user gjacoby126 commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r197821955
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
 ---
@@ -35,20 +35,18 @@
  * during cluster upgrades. However, tenant-owned data such as 
tenant-owned views need to
  * be copied. This WALEntryFilter will only allow tenant-owned rows in 
SYSTEM.CATALOG to
  * be replicated. Data from all other tables is automatically passed. It 
will also copy
- * child links in SYSTEM.CATALOG that are globally-owned but point to 
tenant-owned views.
+ * child links in SYSTEM.CHILD_LINK that are globally-owned but point to 
tenant-owned views.
  *
  */
 public class SystemCatalogWALEntryFilter implements WALEntryFilter {
 
-  private static byte[] CHILD_TABLE_BYTES =
-  new byte[]{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
   @Override
   public WAL.Entry filter(WAL.Entry entry) {
 
-//if the WAL.Entry's table isn't System.Catalog, it auto-passes this 
filter
+//if the WAL.Entry's table isn't System.Catalog or System.Child_Link, 
it auto-passes this filter
 //TODO: when Phoenix drops support for pre-1.3 versions of HBase, redo 
as a WALCellFilter
-if (!SchemaUtil.isMetaTable(entry.getKey().getTablename().getName())){
+byte[] tableName = entry.getKey().getTablename().getName();
+   if (!SchemaUtil.isMetaTable(tableName) && 
!SchemaUtil.isChildLinkTable(tableName)){
--- End diff --

Would it be safe to turn on normal HBase replication on the new 
System.CHILD_LINK? (That is, is there any unwanted data in System.CHILD_LINK 
that this WALFilter wouldn't copy that normal HBase replication would?)

If normal HBase replication works for System.CHILD_LINK, and all view data 
left in System.Catalog starts with tenant_id, then the logic here can be 
greatly simplified, similar to how it was before PHOENIX-4229


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-02 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192570609
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -388,51 +435,65 @@ public void 
testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws
 } catch (TableNotFoundException ignore) {
 }
 ddl = "DROP TABLE " + fullTableName1;
-validateCannotDropTableWithChildViewsWithoutCascade(conn, 
fullTableName1);
 ddl = "DROP VIEW " + fullViewName2;
 conn.createStatement().execute(ddl);
 ddl = "DROP TABLE " + fullTableName1;
 conn.createStatement().execute(ddl);
 }
 
-
+
 @Test
-public void testDisallowDropOfColumnOnParentTable() throws Exception {
+public void testDropOfColumnOnParentTableInvalidatesView() throws 
Exception {
 Connection conn = DriverManager.getConnection(getUrl());
+String fullTableName = generateUniqueTableName();
+String viewName = generateUniqueViewName();
+splitSystemCatalog(Lists.newArrayList(fullTableName, viewName));
+
 String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + 
tableDDLOptions;
 conn.createStatement().execute(ddl);
-String viewName = "V_" + generateUniqueName();
 ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS 
SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
 conn.createStatement().execute(ddl);
 
-try {
-conn.createStatement().execute("ALTER TABLE " + fullTableName 
+ " DROP COLUMN v1");
-fail();
-} catch (SQLException e) {
-
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+conn.createStatement().execute("ALTER TABLE " + fullTableName + " 
DROP COLUMN v1");
+// TODO see if its possibel to prevent the dropping of a column 
thats required by a child view (for its view where clause)
+// the view should be invalid
--- End diff --

Ok, this sounds like it'll work fine.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-02 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192569643
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -388,51 +435,65 @@ public void 
testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws
 } catch (TableNotFoundException ignore) {
 }
 ddl = "DROP TABLE " + fullTableName1;
-validateCannotDropTableWithChildViewsWithoutCascade(conn, 
fullTableName1);
 ddl = "DROP VIEW " + fullViewName2;
 conn.createStatement().execute(ddl);
 ddl = "DROP TABLE " + fullTableName1;
 conn.createStatement().execute(ddl);
 }
 
-
+
 @Test
-public void testDisallowDropOfColumnOnParentTable() throws Exception {
+public void testDropOfColumnOnParentTableInvalidatesView() throws 
Exception {
 Connection conn = DriverManager.getConnection(getUrl());
+String fullTableName = generateUniqueTableName();
+String viewName = generateUniqueViewName();
+splitSystemCatalog(Lists.newArrayList(fullTableName, viewName));
+
 String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + 
tableDDLOptions;
 conn.createStatement().execute(ddl);
-String viewName = "V_" + generateUniqueName();
 ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS 
SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
 conn.createStatement().execute(ddl);
 
-try {
-conn.createStatement().execute("ALTER TABLE " + fullTableName 
+ " DROP COLUMN v1");
-fail();
-} catch (SQLException e) {
-
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+conn.createStatement().execute("ALTER TABLE " + fullTableName + " 
DROP COLUMN v1");
+// TODO see if its possibel to prevent the dropping of a column 
thats required by a child view (for its view where clause)
+// the view should be invalid
--- End diff --

@JamesRTaylor 

We don't need to do the checkAndPut on the header row of the base table as 
this will block clients making changes to different columns. We only need to 
ensure that if multiple clients are making changes to the same column only one 
of them is allowed to make the change. So we can use the rowkey of the column 
row. We could also do the checkAndPut on the new SYSTEM.CHILD_LINK table 
instead of SYSTEM.CATALOG. This would be done from MetadataClient to ensure 
that no other client can make a conflicting change before we call addColumn or 
dropColumn.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192460500
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -388,51 +435,65 @@ public void 
testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws
 } catch (TableNotFoundException ignore) {
 }
 ddl = "DROP TABLE " + fullTableName1;
-validateCannotDropTableWithChildViewsWithoutCascade(conn, 
fullTableName1);
 ddl = "DROP VIEW " + fullViewName2;
 conn.createStatement().execute(ddl);
 ddl = "DROP TABLE " + fullTableName1;
 conn.createStatement().execute(ddl);
 }
 
-
+
 @Test
-public void testDisallowDropOfColumnOnParentTable() throws Exception {
+public void testDropOfColumnOnParentTableInvalidatesView() throws 
Exception {
 Connection conn = DriverManager.getConnection(getUrl());
+String fullTableName = generateUniqueTableName();
+String viewName = generateUniqueViewName();
+splitSystemCatalog(Lists.newArrayList(fullTableName, viewName));
+
 String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + 
tableDDLOptions;
 conn.createStatement().execute(ddl);
-String viewName = "V_" + generateUniqueName();
 ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS 
SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
 conn.createStatement().execute(ddl);
 
-try {
-conn.createStatement().execute("ALTER TABLE " + fullTableName 
+ " DROP COLUMN v1");
-fail();
-} catch (SQLException e) {
-
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+conn.createStatement().execute("ALTER TABLE " + fullTableName + " 
DROP COLUMN v1");
+// TODO see if its possibel to prevent the dropping of a column 
thats required by a child view (for its view where clause)
+// the view should be invalid
--- End diff --

@JamesRTaylor 

I think we can avoid this while dropping a column. However while adding a 
column as the view and base table could be on different region servers we need 
to detect and prevent a conflicting add column change happening at the same 
time. I couldn't think of a better way to prevent this other than doing a 
checkAndPut. 
If we do the checkAndPut in MetadataClient to ensure that only one client 
is able to add the same column at the same time and then make the rpc to the 
server to add the column, do you think think that would scale?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192318659
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java ---
@@ -156,14 +209,14 @@ public String toString() {
 return (familyName == null ? "" : familyName.toString() + 
QueryConstants.NAME_SEPARATOR) + name.toString();
 }
 
-@Override
-public int hashCode() {
-final int prime = 31;
-int result = 1;
-result = prime * result + ((familyName == null) ? 0 : 
familyName.hashCode());
-result = prime * result + ((name == null) ? 0 : name.hashCode());
-return result;
-}
+   @Override
+   public int hashCode() {
+   final int prime = 31;
+   int result = 1;
+   result = prime * result + ((familyName == null) ? 0 : 
familyName.hashCode());
+   result = prime * result + ((name == null) ? 0 : name.hashCode());
+   return result;
+   }
--- End diff --

Fixed


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192318109
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
 ---
@@ -0,0 +1,113 @@
+/**
+ * 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.coprocessor;
+
+import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static 
org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.compile.CreateTableCompiler;
+import org.apache.phoenix.compile.ExpressionCompiler;
+import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.compile.WhereCompiler;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.MetaDataUtil;
+
+import com.google.common.collect.Lists;
+
+
+public class WhereConstantParser {
+
+static PTable addViewInfoToPColumnsIfNeeded(PTable view) throws 
SQLException {
+   boolean[] viewColumnConstantsMatched = new 
boolean[view.getColumns().size()];
+byte[][] viewColumnConstantsToBe = new 
byte[view.getColumns().size()][];
+if (view.getViewStatement() == null) {
+   return view;
+}
+SelectStatement select = new 
SQLParser(view.getViewStatement()).parseQuery();
+ParseNode whereNode = select.getWhere();
+ColumnResolver resolver = FromCompiler.getResolver(new 
TableRef(view));
+StatementContext context = new StatementContext(new 
PhoenixStatement(getConnectionlessConnection()), resolver);
+Expression expression = null;
+try {
+   expression = WhereCompiler.compile(context, whereNode);
+}
+catch (ColumnNotFoundException e) {
+   // if we could not find a column used in the view statement 
(which means its was dropped)
+   // this view is not valid any more
+   return null;
+}
+CreateTableCompiler.ViewWhereExpressionVisitor visitor =
+new CreateTableCompiler.ViewWhereExpressionVisitor(view, 
viewColumnConstantsToBe);
+expression.accept(visitor);
+
+BitSet isViewColumnReferencedToBe = new 
BitSet(view.getColumns().size());
+// Used to track column references in a view
+ExpressionCompiler expressionCompiler = new 
CreateTableCompiler.ColumnTrackingExpressionCompiler(context, 
isViewColumnReferencedToBe);
+whereNode.accept(expressionCompiler);
+
+List result = Lists.newArrayList();
+for (PColumn column : PTableImpl.getColumnsToClone(view)) {
+   boolean isViewReferenced = 
isViewColumnReferencedToBe.get(column.getPosition());
+   if ( (visitor.isUpdatable() || 
view.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(view)).equals(column))
 
+   && 
viewColumnConstantsToBe[column.getPosition()] != null) {
+   result.add(new PColumnImpl(column, 
viewColumnConstantsToBe[column.getPosition()], isViewReferenced));
+   

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192318056
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnNameTrackingExpressionCompiler.java
 ---
@@ -0,0 +1,46 @@
+/*
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.StatelessTraverseAllParseNodeVisitor;
+
+import com.google.common.collect.Lists;
+
+public class ColumnNameTrackingExpressionCompiler extends 
StatelessTraverseAllParseNodeVisitor {
+
+   private List dataColumnNames = 
Lists.newArrayListWithExpectedSize(10);
+
+public void reset() {
+this.getDataColumnNames().clear();
+}
+
+   @Override
+public Void visit(ColumnParseNode node) throws SQLException {
+   getDataColumnNames().add(node.getName());
+return null;
+}
+   
+   public List getDataColumnNames() {
+   return dataColumnNames;
+   }
+
+}
--- End diff --

Fixed.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192317572
  
--- Diff: 
phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java ---
@@ -522,7 +523,8 @@ public void 
helpTestDroppingIndexedColDropsViewIndex(boolean isMultiTenant) thro
 byte[] cq = column.getColumnQualifierBytes();
 // there should be a single row belonging to VIEWINDEX2 
 assertNotNull(viewIndex2 + " row is missing", 
result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, cq));
-assertNull(results.next());
+// TODO enable this after we drop view indexes than need a 
dropped column 
+//assertNull(results.next());
--- End diff --

This test now passes without commenting the assertNull.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192317466
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1457,28 +1761,69 @@ private static void getSchemaTableNames(Mutation 
row, byte[][] schemaTableNames)
 schemaTableNames[2] = tName;
 }
 }
-
+
 @Override
 public void createTable(RpcController controller, CreateTableRequest 
request,
 RpcCallback done) {
 MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
 byte[][] rowKeyMetaData = new byte[3][];
 byte[] schemaName = null;
 byte[] tableName = null;
+String fullTableName = SchemaUtil.getTableName(schemaName, 
tableName);
 try {
 int clientVersion = request.getClientVersion();
 List tableMetadata = 
ProtobufUtil.getMutations(request);
 MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, 
rowKeyMetaData);
 byte[] tenantIdBytes = 
rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 schemaName = 
rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 tableName = 
rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+// TODO before creating a table we need to see if the table 
was previously created and then dropped
+// and clean up any parent->child links or child views
 boolean isNamespaceMapped = 
MetaDataUtil.isNameSpaceMapped(tableMetadata, GenericKeyValueBuilder.INSTANCE,
 new ImmutableBytesWritable());
 final IndexType indexType = 
MetaDataUtil.getIndexType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
 new ImmutableBytesWritable());
+byte[] parentTenantId = null;
 byte[] parentSchemaName = null;
 byte[] parentTableName = null;
 PTableType tableType = 
MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new 
ImmutableBytesWritable());
+ViewType viewType = MetaDataUtil.getViewType(tableMetadata, 
GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
+
+// Here we are passed the parent's columns to add to a view, 
PHOENIX-3534 allows for a splittable
+// System.Catalog thus we only store the columns that are new 
to the view, not the parents columns,
+// thus here we remove everything that is ORDINAL.POSITION <= 
baseColumnCount and update the
+// ORDINAL.POSITIONS to be shifted accordingly.
--- End diff --

I filed PHOENIX-4767 to remove the dedup code. We can stop sending the 
parent column metadata in the same release. 


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192317262
  
--- Diff: 
phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java ---
@@ -1253,7 +1253,9 @@ public void testUnknownColumnInPKConstraint() throws 
Exception {
 }
 }
 
-
+
+// see PHOENIX-3534, now tables can have duplicate columns and they 
are removed implicitly
--- End diff --

This test passes, I have remove the ignore annotation.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192316117
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 ---
@@ -2967,6 +2982,11 @@ protected PhoenixConnection 
upgradeSystemCatalogIfRequired(PhoenixConnection met
 HTableDescriptor.SPLIT_POLICY + "='" + 
SystemStatsSplitPolicy.class.getName() +"'"
 );
 }
+// TODO set the version for which the following upgrade code runs 
correct
+if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0) {
--- End diff --

Done


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192316080
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java 
---
@@ -479,179 +498,355 @@ private void addTenantIdFilter(StringBuilder buf, 
String tenantIdPattern) {
 private static void appendConjunction(StringBuilder buf) {
 buf.append(buf.length() == 0 ? "" : " and ");
 }
-
-@Override
+
+private static final PColumnImpl TENANT_ID_COLUMN = new 
PColumnImpl(PNameFactory.newName(TENANT_ID),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl TABLE_SCHEM_COLUMN = new 
PColumnImpl(PNameFactory.newName(TABLE_SCHEM),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl TABLE_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(TABLE_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl COLUMN_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl DATA_TYPE_COLUMN = new 
PColumnImpl(PNameFactory.newName(DATA_TYPE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl TYPE_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(TYPE_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(TYPE_NAME), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl COLUMN_SIZE_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, COLUMN_SIZE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl BUFFER_LENGTH_COLUMN = new 
PColumnImpl(PNameFactory.newName(BUFFER_LENGTH),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(BUFFER_LENGTH), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl DECIMAL_DIGITS_COLUMN = new 
PColumnImpl(PNameFactory.newName(DECIMAL_DIGITS),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
DECIMAL_DIGITS_BYTES, HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl NUM_PREC_RADIX_COLUMN = new 
PColumnImpl(PNameFactory.newName(NUM_PREC_RADIX),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(NUM_PREC_RADIX), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl NULLABLE_COLUMN = new 
PColumnImpl(PNameFactory.newName(NULLABLE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, NULLABLE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl REMARKS_COLUMN = new 
PColumnImpl(PNameFactory.newName(REMARKS),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(REMARKS), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl COLUMN_DEF_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_DEF),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192316026
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java 
---
@@ -293,6 +293,7 @@ public SQLException newException(SQLExceptionInfo info) 
{
 
 SEQUENCE_NOT_CASTABLE_TO_AUTO_PARTITION_ID_COLUMN(1086, "44A17", 
"Sequence Value not castable to auto-partition id column"),
 CANNOT_COERCE_AUTO_PARTITION_ID(1087, "44A18", "Auto-partition id 
cannot be coerced"),
+
--- End diff --

Done


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192313898
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java ---
@@ -787,7 +787,7 @@ public MutationPlan compile(UpsertStatement upsert) 
throws SQLException {
 LinkedHashSet updateColumns = 
Sets.newLinkedHashSetWithExpectedSize(nColumns + 1);
 updateColumns.add(new PColumnImpl(
 table.getPKColumns().get(position).getName(), // 
Use first PK column name as we know it won't conflict with others
-null, PVarbinary.INSTANCE, null, null, false, 
position, SortOrder.getDefault(), 0, null, false, null, false, false, null));
+null, PVarbinary.INSTANCE, null, null, false, 
position, SortOrder.getDefault(), 0, null, false, null, false, false, null, 
table.getPKColumns().get(0).getTimestamp()));
--- End diff --

We are using the first timestamp of the first pk column, so its guaranteed 
to be non null.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192309043
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -388,51 +435,65 @@ public void 
testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws
 } catch (TableNotFoundException ignore) {
 }
 ddl = "DROP TABLE " + fullTableName1;
-validateCannotDropTableWithChildViewsWithoutCascade(conn, 
fullTableName1);
 ddl = "DROP VIEW " + fullViewName2;
 conn.createStatement().execute(ddl);
 ddl = "DROP TABLE " + fullTableName1;
 conn.createStatement().execute(ddl);
 }
 
-
+
 @Test
-public void testDisallowDropOfColumnOnParentTable() throws Exception {
+public void testDropOfColumnOnParentTableInvalidatesView() throws 
Exception {
 Connection conn = DriverManager.getConnection(getUrl());
+String fullTableName = generateUniqueTableName();
+String viewName = generateUniqueViewName();
+splitSystemCatalog(Lists.newArrayList(fullTableName, viewName));
+
 String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + 
tableDDLOptions;
 conn.createStatement().execute(ddl);
-String viewName = "V_" + generateUniqueName();
 ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS 
SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
 conn.createStatement().execute(ddl);
 
-try {
-conn.createStatement().execute("ALTER TABLE " + fullTableName 
+ " DROP COLUMN v1");
-fail();
-} catch (SQLException e) {
-
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+conn.createStatement().execute("ALTER TABLE " + fullTableName + " 
DROP COLUMN v1");
+// TODO see if its possibel to prevent the dropping of a column 
thats required by a child view (for its view where clause)
+// the view should be invalid
--- End diff --

I think we should avoid needing any kind of locking (including a 
checkAndPut). The scaling issues we ran into were caused by contention on a 
lock for the parent. The same thing could happen again with a checkAndPut 
(which is just locking the row during the check). I'd err on the side of 
scalability and have a view be invalidated if its parent is deleted. I think 
that's a more scalable solution.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192308674
  
--- Diff: 
phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
 ---
@@ -0,0 +1,299 @@
+package org.apache.phoenix.coprocessor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * 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.
+ */
+public class MetaDataEndpointImplTest extends ParallelStatsDisabledIT {
--- End diff --

I think we should keep the OrphanCleaner code. The idea is that failed 
deletions of metadata are transparent. They shouldn't block creation of new 
tables. We should also cleanup on compaction.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192308276
  
--- Diff: 
phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
 ---
@@ -0,0 +1,299 @@
+package org.apache.phoenix.coprocessor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * 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.
+ */
+public class MetaDataEndpointImplTest extends ParallelStatsDisabledIT {
--- End diff --

I removed the OrphanCleaner code that was there previously and I create a 
PHOENIX-3534 to handle cleanup of child view metadata for a parent that was 
dropped during compaction. Instead of having additional clean up code, I think 
we should just detect that we are trying to re-create a table that was dropped 
whose child view metadata wasn't cleaned up and then throw an exception.  


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192305732
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java 
---
@@ -479,179 +498,355 @@ private void addTenantIdFilter(StringBuilder buf, 
String tenantIdPattern) {
 private static void appendConjunction(StringBuilder buf) {
 buf.append(buf.length() == 0 ? "" : " and ");
 }
-
-@Override
+
+private static final PColumnImpl TENANT_ID_COLUMN = new 
PColumnImpl(PNameFactory.newName(TENANT_ID),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl TABLE_SCHEM_COLUMN = new 
PColumnImpl(PNameFactory.newName(TABLE_SCHEM),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl TABLE_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(TABLE_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl COLUMN_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl DATA_TYPE_COLUMN = new 
PColumnImpl(PNameFactory.newName(DATA_TYPE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl TYPE_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(TYPE_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(TYPE_NAME), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl COLUMN_SIZE_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, COLUMN_SIZE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl BUFFER_LENGTH_COLUMN = new 
PColumnImpl(PNameFactory.newName(BUFFER_LENGTH),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(BUFFER_LENGTH), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl DECIMAL_DIGITS_COLUMN = new 
PColumnImpl(PNameFactory.newName(DECIMAL_DIGITS),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
DECIMAL_DIGITS_BYTES, HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl NUM_PREC_RADIX_COLUMN = new 
PColumnImpl(PNameFactory.newName(NUM_PREC_RADIX),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(NUM_PREC_RADIX), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl NULLABLE_COLUMN = new 
PColumnImpl(PNameFactory.newName(NULLABLE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, NULLABLE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl REMARKS_COLUMN = new 
PColumnImpl(PNameFactory.newName(REMARKS),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(REMARKS), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl COLUMN_DEF_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_DEF),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-06-01 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r192305114
  
--- Diff: 
phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 ---
@@ -1202,33 +1202,44 @@ private void 
testUseStatsForParallelizationOnSaltedTable(boolean useStatsFlag, b
 assertEquals("B", rs.getString(1));
 }
 
-   @Test
-   public void testUseStatsForParallelizationProperyOnViewIndex() throws 
SQLException {
-   String tableName = generateUniqueName();
-   String viewName = generateUniqueName();
-   String tenantViewName = generateUniqueName();
-   String viewIndexName = generateUniqueName();
-   boolean useStats = !DEFAULT_USE_STATS_FOR_PARALLELIZATION;
-   try (Connection conn = DriverManager.getConnection(getUrl())) {
-   conn.createStatement()
-   .execute("create table " + tableName
-   + "(tenantId CHAR(15) 
NOT NULL, pk1 integer NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
-   + "(tenantId, pk1)) 
MULTI_TENANT=true");
-   try (Connection tenantConn = 
getTenantConnection("tenant1")) {
-   conn.createStatement().execute("CREATE VIEW " + 
viewName + " AS SELECT * FROM " + tableName);
-   conn.createStatement().execute("CREATE INDEX " 
+ viewIndexName + " on " + viewName + " (v) ");
-   tenantConn.createStatement().execute("CREATE 
VIEW " + tenantViewName + " AS SELECT * FROM " + viewName);
-   conn.createStatement()
-   .execute("ALTER TABLE " + 
tableName + " set USE_STATS_FOR_PARALLELIZATION=" + useStats);
-   // fetch the latest view ptable 
-   PhoenixRuntime.getTableNoCache(tenantConn, 
viewName);
-   PhoenixConnection phxConn = 
conn.unwrap(PhoenixConnection.class);
-   PTable viewIndex = phxConn.getTable(new 
PTableKey(phxConn.getTenantId(), viewIndexName));
-   assertEquals("USE_STATS_FOR_PARALLELIZATION 
property set incorrectly", useStats,
-   PhoenixConfigurationUtil
-   
.getStatsForParallelizationProp(tenantConn.unwrap(PhoenixConnection.class), 
viewIndex));
-   }
-   }
-   }
+@Test
+public void testUseStatsForParallelizationProperyOnViewIndex() throws 
SQLException {
+String tableName = generateUniqueName();
+String viewName = generateUniqueName();
+String tenantViewName = generateUniqueName();
+String viewIndexName = generateUniqueName();
+boolean useStats = !DEFAULT_USE_STATS_FOR_PARALLELIZATION;
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+conn.createStatement()
+.execute("create table " + tableName
++ "(tenantId CHAR(15) NOT NULL, pk1 integer 
NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
++ "(tenantId, pk1)) MULTI_TENANT=true");
+try (Connection tenantConn = getTenantConnection("tenant1")) {
+conn.createStatement().execute("CREATE VIEW " + viewName + 
" AS SELECT * FROM " + tableName);
+conn.createStatement().execute("CREATE INDEX " + 
viewIndexName + " on " + viewName + " (v) ");
+tenantConn.createStatement().execute("CREATE VIEW " + 
tenantViewName + " AS SELECT * FROM " + viewName);
+conn.createStatement()
+.execute("ALTER TABLE " + tableName + " set 
USE_STATS_FOR_PARALLELIZATION=" + useStats);
+// changing a property on a base table does not change the 
property on a view
--- End diff --

In PTable we don't have access to when a table property was updated. In 
combineColumns when creating the PTable of the view if a table property is not 
mutable on a view , the value is set to the same as the base physical table. 
If the table property is mutable on a view we don't know if the value was 
modified in the view or not so its we set the table property value to the one 
in the view. This is different from current behavior where if we change  a 
table property on a base table and a child view hasn't modified the property, 
the change is propagated to the child.  I filed PHOENIX-4763 to fix this as a 
follow-up task.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread churrodog
Github user churrodog commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191948329
  
--- Diff: 
phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java ---
@@ -522,7 +523,8 @@ public void 
helpTestDroppingIndexedColDropsViewIndex(boolean isMultiTenant) thro
 byte[] cq = column.getColumnQualifierBytes();
 // there should be a single row belonging to VIEWINDEX2 
 assertNotNull(viewIndex2 + " row is missing", 
result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, cq));
-assertNull(results.next());
+// TODO enable this after we drop view indexes than need a 
dropped column 
+//assertNull(results.next());
--- End diff --

what about this TODO?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread churrodog
Github user churrodog commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191948951
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
 ---
@@ -0,0 +1,113 @@
+/**
+ * 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.coprocessor;
+
+import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static 
org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.compile.CreateTableCompiler;
+import org.apache.phoenix.compile.ExpressionCompiler;
+import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.compile.WhereCompiler;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.MetaDataUtil;
+
+import com.google.common.collect.Lists;
+
+
+public class WhereConstantParser {
+
+static PTable addViewInfoToPColumnsIfNeeded(PTable view) throws 
SQLException {
+   boolean[] viewColumnConstantsMatched = new 
boolean[view.getColumns().size()];
+byte[][] viewColumnConstantsToBe = new 
byte[view.getColumns().size()][];
+if (view.getViewStatement() == null) {
+   return view;
+}
+SelectStatement select = new 
SQLParser(view.getViewStatement()).parseQuery();
+ParseNode whereNode = select.getWhere();
+ColumnResolver resolver = FromCompiler.getResolver(new 
TableRef(view));
+StatementContext context = new StatementContext(new 
PhoenixStatement(getConnectionlessConnection()), resolver);
+Expression expression = null;
+try {
+   expression = WhereCompiler.compile(context, whereNode);
+}
+catch (ColumnNotFoundException e) {
+   // if we could not find a column used in the view statement 
(which means its was dropped)
+   // this view is not valid any more
+   return null;
+}
+CreateTableCompiler.ViewWhereExpressionVisitor visitor =
+new CreateTableCompiler.ViewWhereExpressionVisitor(view, 
viewColumnConstantsToBe);
+expression.accept(visitor);
+
+BitSet isViewColumnReferencedToBe = new 
BitSet(view.getColumns().size());
+// Used to track column references in a view
+ExpressionCompiler expressionCompiler = new 
CreateTableCompiler.ColumnTrackingExpressionCompiler(context, 
isViewColumnReferencedToBe);
+whereNode.accept(expressionCompiler);
+
+List result = Lists.newArrayList();
+for (PColumn column : PTableImpl.getColumnsToClone(view)) {
+   boolean isViewReferenced = 
isViewColumnReferencedToBe.get(column.getPosition());
+   if ( (visitor.isUpdatable() || 
view.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(view)).equals(column))
 
+   && 
viewColumnConstantsToBe[column.getPosition()] != null) {
+   result.add(new PColumnImpl(column, 
viewColumnConstantsToBe[column.getPosition()], isViewReferenced));
+   

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread churrodog
Github user churrodog commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191948812
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java ---
@@ -787,7 +787,7 @@ public MutationPlan compile(UpsertStatement upsert) 
throws SQLException {
 LinkedHashSet updateColumns = 
Sets.newLinkedHashSetWithExpectedSize(nColumns + 1);
 updateColumns.add(new PColumnImpl(
 table.getPKColumns().get(position).getName(), // 
Use first PK column name as we know it won't conflict with others
-null, PVarbinary.INSTANCE, null, null, false, 
position, SortOrder.getDefault(), 0, null, false, null, false, false, null));
+null, PVarbinary.INSTANCE, null, null, false, 
position, SortOrder.getDefault(), 0, null, false, null, false, false, null, 
table.getPKColumns().get(0).getTimestamp()));
--- End diff --

are we guaranteed to get a non null column back?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191948821
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -388,51 +435,65 @@ public void 
testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws
 } catch (TableNotFoundException ignore) {
 }
 ddl = "DROP TABLE " + fullTableName1;
-validateCannotDropTableWithChildViewsWithoutCascade(conn, 
fullTableName1);
 ddl = "DROP VIEW " + fullViewName2;
 conn.createStatement().execute(ddl);
 ddl = "DROP TABLE " + fullTableName1;
 conn.createStatement().execute(ddl);
 }
 
-
+
 @Test
-public void testDisallowDropOfColumnOnParentTable() throws Exception {
+public void testDropOfColumnOnParentTableInvalidatesView() throws 
Exception {
 Connection conn = DriverManager.getConnection(getUrl());
+String fullTableName = generateUniqueTableName();
+String viewName = generateUniqueViewName();
+splitSystemCatalog(Lists.newArrayList(fullTableName, viewName));
+
 String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + 
tableDDLOptions;
 conn.createStatement().execute(ddl);
-String viewName = "V_" + generateUniqueName();
 ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS 
SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
 conn.createStatement().execute(ddl);
 
-try {
-conn.createStatement().execute("ALTER TABLE " + fullTableName 
+ " DROP COLUMN v1");
-fail();
-} catch (SQLException e) {
-
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+conn.createStatement().execute("ALTER TABLE " + fullTableName + " 
DROP COLUMN v1");
+// TODO see if its possibel to prevent the dropping of a column 
thats required by a child view (for its view where clause)
+// the view should be invalid
--- End diff --

I was thinking of handling the race condition by writing a cell using 
checkAndPut with the column name being mutated (added or dropped) to the header 
row of the base table. When a view is create the columns used in the view where 
clause will also do a checkAndPut on this cell in the base table. If this is 
happening concurrently only of the clients will succeed (either the drop 
column, or the view creation). 

If a DROP TABLE cascade is issued then the base table (with child views) is 
dropped. When any child views are resolved and we see the base table does not 
exists we throw a TableNotFoundException. If the DROP VIEW is issued without 
CASCADE and there are child views the statement fails with a not allowed to 
mutate exception. If a drop table without cascade and a create view happens 
concurrently we could create the view even though the base table would have 
been dropped, but the next time the view is resolved you would get a 
TableNotFoundException.   


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread churrodog
Github user churrodog commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191948510
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnNameTrackingExpressionCompiler.java
 ---
@@ -0,0 +1,46 @@
+/*
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.StatelessTraverseAllParseNodeVisitor;
+
+import com.google.common.collect.Lists;
+
+public class ColumnNameTrackingExpressionCompiler extends 
StatelessTraverseAllParseNodeVisitor {
+
+   private List dataColumnNames = 
Lists.newArrayListWithExpectedSize(10);
+
+public void reset() {
+this.getDataColumnNames().clear();
+}
+
+   @Override
+public Void visit(ColumnParseNode node) throws SQLException {
+   getDataColumnNames().add(node.getName());
+return null;
+}
+   
+   public List getDataColumnNames() {
+   return dataColumnNames;
+   }
+
+}
--- End diff --

Nit, but indentation seems off for this class. 


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191944884
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -388,51 +435,65 @@ public void 
testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws
 } catch (TableNotFoundException ignore) {
 }
 ddl = "DROP TABLE " + fullTableName1;
-validateCannotDropTableWithChildViewsWithoutCascade(conn, 
fullTableName1);
 ddl = "DROP VIEW " + fullViewName2;
 conn.createStatement().execute(ddl);
 ddl = "DROP TABLE " + fullTableName1;
 conn.createStatement().execute(ddl);
 }
 
-
+
 @Test
-public void testDisallowDropOfColumnOnParentTable() throws Exception {
+public void testDropOfColumnOnParentTableInvalidatesView() throws 
Exception {
 Connection conn = DriverManager.getConnection(getUrl());
+String fullTableName = generateUniqueTableName();
+String viewName = generateUniqueViewName();
+splitSystemCatalog(Lists.newArrayList(fullTableName, viewName));
+
 String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + 
tableDDLOptions;
 conn.createStatement().execute(ddl);
-String viewName = "V_" + generateUniqueName();
 ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS 
SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
 conn.createStatement().execute(ddl);
 
-try {
-conn.createStatement().execute("ALTER TABLE " + fullTableName 
+ " DROP COLUMN v1");
-fail();
-} catch (SQLException e) {
-
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+conn.createStatement().execute("ALTER TABLE " + fullTableName + " 
DROP COLUMN v1");
+// TODO see if its possibel to prevent the dropping of a column 
thats required by a child view (for its view where clause)
+// the view should be invalid
--- End diff --

Seems like there'd be an inherent race condition, but if not, I suppose 
keeping the same behavior is fine. What about preventing the drop of a table 
with child views?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191944295
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -2227,35 +2551,36 @@ private MetaDataMutationResult doDropTable(byte[] 
key, byte[] tenantId, byte[] s
 // in 0.94.4, thus if we try to use it here we can no longer 
use the 0.94.2 version
 // of the client.
 Delete delete = new Delete(indexKey, clientTimeStamp);
-rowsToDelete.add(delete);
-acquireLock(region, indexKey, locks);
+catalogMutations.add(delete);
 MetaDataMutationResult result =
 doDropTable(indexKey, tenantId, schemaName, indexName, 
tableName, PTableType.INDEX,
-rowsToDelete, invalidateList, locks, 
tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
+catalogMutations, childLinkMutations, 
invalidateList, tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
 if (result.getMutationCode() != 
MutationCode.TABLE_ALREADY_EXISTS) {
 return result;
 }
 }
 
+// no need to pass sharedTablesToDelete back to the client as they 
deletion of these tables
+// is already handled in MetadataClient.dropTable
--- End diff --

We still pass sharedTablesToDelete back to the client when calling 
dropColumn, so that we drop the local or view index data I think. We don't need 
to pass sharedTablesToDelete back to the client when dropping a table.  


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191942250
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -2227,35 +2551,36 @@ private MetaDataMutationResult doDropTable(byte[] 
key, byte[] tenantId, byte[] s
 // in 0.94.4, thus if we try to use it here we can no longer 
use the 0.94.2 version
 // of the client.
 Delete delete = new Delete(indexKey, clientTimeStamp);
-rowsToDelete.add(delete);
-acquireLock(region, indexKey, locks);
+catalogMutations.add(delete);
 MetaDataMutationResult result =
 doDropTable(indexKey, tenantId, schemaName, indexName, 
tableName, PTableType.INDEX,
-rowsToDelete, invalidateList, locks, 
tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
+catalogMutations, childLinkMutations, 
invalidateList, tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
 if (result.getMutationCode() != 
MutationCode.TABLE_ALREADY_EXISTS) {
 return result;
 }
 }
 
+// no need to pass sharedTablesToDelete back to the client as they 
deletion of these tables
+// is already handled in MetadataClient.dropTable
--- End diff --

Not sure if this is handled differently now, but we passed this back 
because I believe we don't know on the client all of the physical index tables 
to delete. I think we have a test for this.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191941040
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1995,36 +2287,46 @@ public void dropTable(RpcController controller, 
DropTableRequest request,
 
 try {
 List tableMetadata = 
ProtobufUtil.getMutations(request);
+List childLinkMutations = Lists.newArrayList();
 MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, 
rowKeyMetaData);
 byte[] tenantIdBytes = 
rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 schemaName = 
rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 tableName = 
rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+PTableType 
pTableType=PTableType.fromSerializedValue(tableType);
 // Disallow deletion of a system table
-if (tableType.equals(PTableType.SYSTEM.getSerializedValue())) {
+if (pTableType == PTableType.SYSTEM) {
 
builder.setReturnCode(MetaDataProtos.MutationCode.UNALLOWED_TABLE_MUTATION);
 
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
 done.run(builder.build());
 return;
 }
+
 List tableNamesToDelete = Lists.newArrayList();
 List sharedTablesToDelete = 
Lists.newArrayList();
-// No need to lock parent table for views
-byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
-byte[] lockTableName = parentTableName == null || 
tableType.equals(PTableType.VIEW.getSerializedValue()) ? tableName : 
parentTableName;
-byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, 
schemaName, lockTableName);
-byte[] key =
-parentTableName == null ? lockKey : 
SchemaUtil.getTableKey(tenantIdBytes,
-schemaName, tableName);
+
+byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, 
schemaName, tableName);
 Region region = env.getRegion();
-MetaDataMutationResult result = checkTableKeyInRegion(key, 
region);
+MetaDataMutationResult result = checkTableKeyInRegion(lockKey, 
region);
 if (result != null) {
 done.run(MetaDataMutationResult.toProto(result));
 return;
 }
-PTableType 
ptableType=PTableType.fromSerializedValue(tableType);
+
+byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
+byte[] parentLockKey = null;
+// No need to lock parent table for views
+if (parentTableName != null && pTableType != PTableType.VIEW) {
+parentLockKey = SchemaUtil.getTableKey(tenantIdBytes, 
schemaName, parentTableName);
--- End diff --

We only lock the parent for indexes.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191940463
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1995,36 +2287,46 @@ public void dropTable(RpcController controller, 
DropTableRequest request,
 
 try {
 List tableMetadata = 
ProtobufUtil.getMutations(request);
+List childLinkMutations = Lists.newArrayList();
 MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, 
rowKeyMetaData);
 byte[] tenantIdBytes = 
rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 schemaName = 
rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 tableName = 
rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+PTableType 
pTableType=PTableType.fromSerializedValue(tableType);
 // Disallow deletion of a system table
-if (tableType.equals(PTableType.SYSTEM.getSerializedValue())) {
+if (pTableType == PTableType.SYSTEM) {
 
builder.setReturnCode(MetaDataProtos.MutationCode.UNALLOWED_TABLE_MUTATION);
 
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
 done.run(builder.build());
 return;
 }
+
 List tableNamesToDelete = Lists.newArrayList();
 List sharedTablesToDelete = 
Lists.newArrayList();
-// No need to lock parent table for views
-byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
-byte[] lockTableName = parentTableName == null || 
tableType.equals(PTableType.VIEW.getSerializedValue()) ? tableName : 
parentTableName;
-byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, 
schemaName, lockTableName);
-byte[] key =
-parentTableName == null ? lockKey : 
SchemaUtil.getTableKey(tenantIdBytes,
-schemaName, tableName);
+
+byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, 
schemaName, tableName);
 Region region = env.getRegion();
-MetaDataMutationResult result = checkTableKeyInRegion(key, 
region);
+MetaDataMutationResult result = checkTableKeyInRegion(lockKey, 
region);
 if (result != null) {
 done.run(MetaDataMutationResult.toProto(result));
 return;
 }
-PTableType 
ptableType=PTableType.fromSerializedValue(tableType);
+
+byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
+byte[] parentLockKey = null;
+// No need to lock parent table for views
+if (parentTableName != null && pTableType != PTableType.VIEW) {
+parentLockKey = SchemaUtil.getTableKey(tenantIdBytes, 
schemaName, parentTableName);
--- End diff --

Shouldn't need this parentLockKey any longer, yes? Or is this only for 
indexes?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191939788
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -1457,28 +1761,69 @@ private static void getSchemaTableNames(Mutation 
row, byte[][] schemaTableNames)
 schemaTableNames[2] = tName;
 }
 }
-
+
 @Override
 public void createTable(RpcController controller, CreateTableRequest 
request,
 RpcCallback done) {
 MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
 byte[][] rowKeyMetaData = new byte[3][];
 byte[] schemaName = null;
 byte[] tableName = null;
+String fullTableName = SchemaUtil.getTableName(schemaName, 
tableName);
 try {
 int clientVersion = request.getClientVersion();
 List tableMetadata = 
ProtobufUtil.getMutations(request);
 MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, 
rowKeyMetaData);
 byte[] tenantIdBytes = 
rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 schemaName = 
rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 tableName = 
rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+// TODO before creating a table we need to see if the table 
was previously created and then dropped
+// and clean up any parent->child links or child views
 boolean isNamespaceMapped = 
MetaDataUtil.isNameSpaceMapped(tableMetadata, GenericKeyValueBuilder.INSTANCE,
 new ImmutableBytesWritable());
 final IndexType indexType = 
MetaDataUtil.getIndexType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
 new ImmutableBytesWritable());
+byte[] parentTenantId = null;
 byte[] parentSchemaName = null;
 byte[] parentTableName = null;
 PTableType tableType = 
MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new 
ImmutableBytesWritable());
+ViewType viewType = MetaDataUtil.getViewType(tableMetadata, 
GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
+
+// Here we are passed the parent's columns to add to a view, 
PHOENIX-3534 allows for a splittable
+// System.Catalog thus we only store the columns that are new 
to the view, not the parents columns,
+// thus here we remove everything that is ORDINAL.POSITION <= 
baseColumnCount and update the
+// ORDINAL.POSITIONS to be shifted accordingly.
--- End diff --

Important to file and reference a JIRA here to remove the dedup code once 
clients have been upgraded to the release in which we no longer send the 
duplicate information. Can we stop sending the duplicate info in the same 
release that SYSTEM.CATALOG becomes splittable? Seems like yes.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191939222
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -388,51 +435,65 @@ public void 
testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws
 } catch (TableNotFoundException ignore) {
 }
 ddl = "DROP TABLE " + fullTableName1;
-validateCannotDropTableWithChildViewsWithoutCascade(conn, 
fullTableName1);
 ddl = "DROP VIEW " + fullViewName2;
 conn.createStatement().execute(ddl);
 ddl = "DROP TABLE " + fullTableName1;
 conn.createStatement().execute(ddl);
 }
 
-
+
 @Test
-public void testDisallowDropOfColumnOnParentTable() throws Exception {
+public void testDropOfColumnOnParentTableInvalidatesView() throws 
Exception {
 Connection conn = DriverManager.getConnection(getUrl());
+String fullTableName = generateUniqueTableName();
+String viewName = generateUniqueViewName();
+splitSystemCatalog(Lists.newArrayList(fullTableName, viewName));
+
 String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + 
tableDDLOptions;
 conn.createStatement().execute(ddl);
-String viewName = "V_" + generateUniqueName();
 ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS 
SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
 conn.createStatement().execute(ddl);
 
-try {
-conn.createStatement().execute("ALTER TABLE " + fullTableName 
+ " DROP COLUMN v1");
-fail();
-} catch (SQLException e) {
-
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+conn.createStatement().execute("ALTER TABLE " + fullTableName + " 
DROP COLUMN v1");
+// TODO see if its possibel to prevent the dropping of a column 
thats required by a child view (for its view where clause)
+// the view should be invalid
--- End diff --

Previously we used to prevent a base table column from being dropped if the 
column was used in a child view index. 
It's possible to detect this and prevent the column drop even with 
splittable system.catalog if you think we should maintain the existing behavior.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread twdsilva
Github user twdsilva commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191938764
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -586,48 +573,336 @@ public void getTable(RpcController controller, 
GetTableRequest request,
 
builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
 }
 }
-
-if (table.getTimeStamp() != tableTimeStamp) {
+// the PTable of views and indexes on views might get updated 
because a column is added to one of
+// their parents (this won't change the timestamp)
+if (table.getType()!=PTableType.TABLE || table.getTimeStamp() 
!= tableTimeStamp) {
 builder.setTable(PTableImpl.toProto(table));
 }
 done.run(builder.build());
-return;
 } catch (Throwable t) {
 logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
 }
 
+/**
+ * Used to add the columns present the ancestor hierarchy to the 
PTable of the given view or
+ * view index
+ * @param table PTable of the view or view index
+ * @param skipAddingIndexes if true the returned PTable won't include 
indexes
+ * @param skipAddingParentColumns if true the returned PTable won't 
include columns derived from ancestor tables
+ */
+private Pair 
combineColumns(PTable table, long timestamp,
+int clientVersion, boolean skipAddingIndexes, boolean 
skipAddingParentColumns) throws SQLException, IOException {
+boolean hasIndexId = table.getViewIndexId() != null;
+if (table.getType() != PTableType.VIEW && !hasIndexId) {
--- End diff --

We only need to combine columns for a VIEW or an index on a VIEW. I will 
add a comment for this.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191938147
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
 ---
@@ -586,48 +573,336 @@ public void getTable(RpcController controller, 
GetTableRequest request,
 
builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
 }
 }
-
-if (table.getTimeStamp() != tableTimeStamp) {
+// the PTable of views and indexes on views might get updated 
because a column is added to one of
+// their parents (this won't change the timestamp)
+if (table.getType()!=PTableType.TABLE || table.getTimeStamp() 
!= tableTimeStamp) {
 builder.setTable(PTableImpl.toProto(table));
 }
 done.run(builder.build());
-return;
 } catch (Throwable t) {
 logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
 }
 
+/**
+ * Used to add the columns present the ancestor hierarchy to the 
PTable of the given view or
+ * view index
+ * @param table PTable of the view or view index
+ * @param skipAddingIndexes if true the returned PTable won't include 
indexes
+ * @param skipAddingParentColumns if true the returned PTable won't 
include columns derived from ancestor tables
+ */
+private Pair 
combineColumns(PTable table, long timestamp,
+int clientVersion, boolean skipAddingIndexes, boolean 
skipAddingParentColumns) throws SQLException, IOException {
+boolean hasIndexId = table.getViewIndexId() != null;
+if (table.getType() != PTableType.VIEW && !hasIndexId) {
--- End diff --

Just curious - why does the viewIndexId determine whether or not the table 
already exists?


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191936918
  
--- Diff: 
phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
 ---
@@ -0,0 +1,299 @@
+package org.apache.phoenix.coprocessor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * 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.
+ */
+public class MetaDataEndpointImplTest extends ParallelStatsDisabledIT {
--- End diff --

One test that would be useful (and maybe you already have it?) is to create 
a table, create a view (maybe on a different RS). Then drop the table and 
recreate it and the view with the same name but different columns. Make sure 
that the lazy cleanup code cleaned up the left over state correctly.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191936296
  
--- Diff: 
phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java ---
@@ -1253,7 +1253,9 @@ public void testUnknownColumnInPKConstraint() throws 
Exception {
 }
 }
 
-
+
+// see PHOENIX-3534, now tables can have duplicate columns and they 
are removed implicitly
--- End diff --

This should still be detected with an exception thrown.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191935338
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java ---
@@ -156,14 +209,14 @@ public String toString() {
 return (familyName == null ? "" : familyName.toString() + 
QueryConstants.NAME_SEPARATOR) + name.toString();
 }
 
-@Override
-public int hashCode() {
-final int prime = 31;
-int result = 1;
-result = prime * result + ((familyName == null) ? 0 : 
familyName.hashCode());
-result = prime * result + ((name == null) ? 0 : name.hashCode());
-return result;
-}
+   @Override
+   public int hashCode() {
+   final int prime = 31;
+   int result = 1;
+   result = prime * result + ((familyName == null) ? 0 : 
familyName.hashCode());
+   result = prime * result + ((name == null) ? 0 : name.hashCode());
+   return result;
+   }
--- End diff --

Minor nit - various formatting issues. Not sure if there are tabs now or if 
the indenting was wrong before.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191933107
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 ---
@@ -2967,6 +2982,11 @@ protected PhoenixConnection 
upgradeSystemCatalogIfRequired(PhoenixConnection met
 HTableDescriptor.SPLIT_POLICY + "='" + 
SystemStatsSplitPolicy.class.getName() +"'"
 );
 }
+// TODO set the version for which the following upgrade code runs 
correct
+if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0) {
--- End diff --

Yes - just add a MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191932416
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java 
---
@@ -479,179 +498,355 @@ private void addTenantIdFilter(StringBuilder buf, 
String tenantIdPattern) {
 private static void appendConjunction(StringBuilder buf) {
 buf.append(buf.length() == 0 ? "" : " and ");
 }
-
-@Override
+
+private static final PColumnImpl TENANT_ID_COLUMN = new 
PColumnImpl(PNameFactory.newName(TENANT_ID),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl TABLE_SCHEM_COLUMN = new 
PColumnImpl(PNameFactory.newName(TABLE_SCHEM),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl TABLE_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(TABLE_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl COLUMN_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl DATA_TYPE_COLUMN = new 
PColumnImpl(PNameFactory.newName(DATA_TYPE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl TYPE_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(TYPE_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(TYPE_NAME), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl COLUMN_SIZE_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, COLUMN_SIZE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl BUFFER_LENGTH_COLUMN = new 
PColumnImpl(PNameFactory.newName(BUFFER_LENGTH),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(BUFFER_LENGTH), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl DECIMAL_DIGITS_COLUMN = new 
PColumnImpl(PNameFactory.newName(DECIMAL_DIGITS),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
DECIMAL_DIGITS_BYTES, HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl NUM_PREC_RADIX_COLUMN = new 
PColumnImpl(PNameFactory.newName(NUM_PREC_RADIX),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(NUM_PREC_RADIX), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl NULLABLE_COLUMN = new 
PColumnImpl(PNameFactory.newName(NULLABLE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, NULLABLE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl REMARKS_COLUMN = new 
PColumnImpl(PNameFactory.newName(REMARKS),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(REMARKS), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl COLUMN_DEF_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_DEF),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191931341
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java 
---
@@ -479,179 +498,355 @@ private void addTenantIdFilter(StringBuilder buf, 
String tenantIdPattern) {
 private static void appendConjunction(StringBuilder buf) {
 buf.append(buf.length() == 0 ? "" : " and ");
 }
-
-@Override
+
+private static final PColumnImpl TENANT_ID_COLUMN = new 
PColumnImpl(PNameFactory.newName(TENANT_ID),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl TABLE_SCHEM_COLUMN = new 
PColumnImpl(PNameFactory.newName(TABLE_SCHEM),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl TABLE_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(TABLE_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+private static final PColumnImpl COLUMN_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl DATA_TYPE_COLUMN = new 
PColumnImpl(PNameFactory.newName(DATA_TYPE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, DATA_TYPE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl TYPE_NAME_COLUMN = new 
PColumnImpl(PNameFactory.newName(TYPE_NAME),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(TYPE_NAME), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl COLUMN_SIZE_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, COLUMN_SIZE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl BUFFER_LENGTH_COLUMN = new 
PColumnImpl(PNameFactory.newName(BUFFER_LENGTH),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(BUFFER_LENGTH), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl DECIMAL_DIGITS_COLUMN = new 
PColumnImpl(PNameFactory.newName(DECIMAL_DIGITS),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
DECIMAL_DIGITS_BYTES, HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl NUM_PREC_RADIX_COLUMN = new 
PColumnImpl(PNameFactory.newName(NUM_PREC_RADIX),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(NUM_PREC_RADIX), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl NULLABLE_COLUMN = new 
PColumnImpl(PNameFactory.newName(NULLABLE),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, NULLABLE_BYTES, 
HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl REMARKS_COLUMN = new 
PColumnImpl(PNameFactory.newName(REMARKS),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 
PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+   0, null, false, null, false, false, 
Bytes.toBytes(REMARKS), HConstants.LATEST_TIMESTAMP);
+   private static final PColumnImpl COLUMN_DEF_COLUMN = new 
PColumnImpl(PNameFactory.newName(COLUMN_DEF),
+   PNameFactory.newName(TABLE_FAMILY_BYTES), 

[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191927956
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java 
---
@@ -293,6 +293,7 @@ public SQLException newException(SQLExceptionInfo info) 
{
 
 SEQUENCE_NOT_CASTABLE_TO_AUTO_PARTITION_ID_COLUMN(1086, "44A17", 
"Sequence Value not castable to auto-partition id column"),
 CANNOT_COERCE_AUTO_PARTITION_ID(1087, "44A18", "Auto-partition id 
cannot be coerced"),
+
--- End diff --

Revert please.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191926624
  
--- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java 
---
@@ -388,51 +435,65 @@ public void 
testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws
 } catch (TableNotFoundException ignore) {
 }
 ddl = "DROP TABLE " + fullTableName1;
-validateCannotDropTableWithChildViewsWithoutCascade(conn, 
fullTableName1);
 ddl = "DROP VIEW " + fullViewName2;
 conn.createStatement().execute(ddl);
 ddl = "DROP TABLE " + fullTableName1;
 conn.createStatement().execute(ddl);
 }
 
-
+
 @Test
-public void testDisallowDropOfColumnOnParentTable() throws Exception {
+public void testDropOfColumnOnParentTableInvalidatesView() throws 
Exception {
 Connection conn = DriverManager.getConnection(getUrl());
+String fullTableName = generateUniqueTableName();
+String viewName = generateUniqueViewName();
+splitSystemCatalog(Lists.newArrayList(fullTableName, viewName));
+
 String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + 
tableDDLOptions;
 conn.createStatement().execute(ddl);
-String viewName = "V_" + generateUniqueName();
 ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS 
SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
 conn.createStatement().execute(ddl);
 
-try {
-conn.createStatement().execute("ALTER TABLE " + fullTableName 
+ " DROP COLUMN v1");
-fail();
-} catch (SQLException e) {
-
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+conn.createStatement().execute("ALTER TABLE " + fullTableName + " 
DROP COLUMN v1");
+// TODO see if its possibel to prevent the dropping of a column 
thats required by a child view (for its view where clause)
+// the view should be invalid
--- End diff --

I think it's fine to consider the view invalid (i.e. fail any query that 
attempts to use it) if all it's columns can no longer be found. This is pretty 
typical in RDBMS.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-30 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/303#discussion_r191925360
  
--- Diff: 
phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 ---
@@ -1202,33 +1202,44 @@ private void 
testUseStatsForParallelizationOnSaltedTable(boolean useStatsFlag, b
 assertEquals("B", rs.getString(1));
 }
 
-   @Test
-   public void testUseStatsForParallelizationProperyOnViewIndex() throws 
SQLException {
-   String tableName = generateUniqueName();
-   String viewName = generateUniqueName();
-   String tenantViewName = generateUniqueName();
-   String viewIndexName = generateUniqueName();
-   boolean useStats = !DEFAULT_USE_STATS_FOR_PARALLELIZATION;
-   try (Connection conn = DriverManager.getConnection(getUrl())) {
-   conn.createStatement()
-   .execute("create table " + tableName
-   + "(tenantId CHAR(15) 
NOT NULL, pk1 integer NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
-   + "(tenantId, pk1)) 
MULTI_TENANT=true");
-   try (Connection tenantConn = 
getTenantConnection("tenant1")) {
-   conn.createStatement().execute("CREATE VIEW " + 
viewName + " AS SELECT * FROM " + tableName);
-   conn.createStatement().execute("CREATE INDEX " 
+ viewIndexName + " on " + viewName + " (v) ");
-   tenantConn.createStatement().execute("CREATE 
VIEW " + tenantViewName + " AS SELECT * FROM " + viewName);
-   conn.createStatement()
-   .execute("ALTER TABLE " + 
tableName + " set USE_STATS_FOR_PARALLELIZATION=" + useStats);
-   // fetch the latest view ptable 
-   PhoenixRuntime.getTableNoCache(tenantConn, 
viewName);
-   PhoenixConnection phxConn = 
conn.unwrap(PhoenixConnection.class);
-   PTable viewIndex = phxConn.getTable(new 
PTableKey(phxConn.getTenantId(), viewIndexName));
-   assertEquals("USE_STATS_FOR_PARALLELIZATION 
property set incorrectly", useStats,
-   PhoenixConfigurationUtil
-   
.getStatsForParallelizationProp(tenantConn.unwrap(PhoenixConnection.class), 
viewIndex));
-   }
-   }
-   }
+@Test
+public void testUseStatsForParallelizationProperyOnViewIndex() throws 
SQLException {
+String tableName = generateUniqueName();
+String viewName = generateUniqueName();
+String tenantViewName = generateUniqueName();
+String viewIndexName = generateUniqueName();
+boolean useStats = !DEFAULT_USE_STATS_FOR_PARALLELIZATION;
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+conn.createStatement()
+.execute("create table " + tableName
++ "(tenantId CHAR(15) NOT NULL, pk1 integer 
NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
++ "(tenantId, pk1)) MULTI_TENANT=true");
+try (Connection tenantConn = getTenantConnection("tenant1")) {
+conn.createStatement().execute("CREATE VIEW " + viewName + 
" AS SELECT * FROM " + tableName);
+conn.createStatement().execute("CREATE INDEX " + 
viewIndexName + " on " + viewName + " (v) ");
+tenantConn.createStatement().execute("CREATE VIEW " + 
tenantViewName + " AS SELECT * FROM " + viewName);
+conn.createStatement()
+.execute("ALTER TABLE " + tableName + " set 
USE_STATS_FOR_PARALLELIZATION=" + useStats);
+// changing a property on a base table does not change the 
property on a view
--- End diff --

Shouldn't setting a property on the base table impact the view as well? In 
this case, USE_STATS_FOR_PARALLELIZATION only makes sense to set on a physical 
table. I think we only look it up from a physical table as well, so this is 
somewhat moot, but in general, I'd think that setting a property on a base 
table should be seen by it's views if the property has not been set there.


---


[GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ...

2018-05-29 Thread twdsilva
GitHub user twdsilva opened a pull request:

https://github.com/apache/phoenix/pull/303

PHOENIX-3534 Support multi region SYSTEM.CATALOG table


This patch adds two new LinkTypes EXCLUDED_COLUMNS (used to represent a 
column that has been dropped) and VIEW_INDEX_PARENT_TABLE (used to link an 
index on a view to its parent). Views and view indexes no longer store columns 
derived from their ancestors in their metadata. When they are resolved the 
ancestors are looked up and added to the PTable that is returned to the client 
(see combineColumns in MetadataEndpointImpl). The PTable in the server side 
metadata cache only stores the  columns created in the view/view index and not 
derived columns. 
We do not propagate metadata changes made to a parent to all its children.  
While adding  columns to a base table, we no longer lock all the children in 
the view hierarchy, we only validate that the columns being added does not 
conflict with an existing base table column. We also don't lock children while 
dropping a parent table column. When dropping a parent column we also drop any 
view indexes that need the column. This patch does not handle the case when 
there are concurrent changes (eg. adding a conflicting column or creating a new 
view index that requires a parent column that is being dropped). That will be 
handled in a follow-up patch. 
While dropping a parent table, we don't drop all the child views metadata. 
This metadata needs to be cleaned-up (maybe at compaction time?) which will be 
handled in a follow-up patch. 

There are a few test failures I am working through, which I will fix soon 
and update the PR. 
@JamesRTaylor can you please review?

FYI @karanmehta93  @ChinmaySKulkarni 


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/twdsilva/phoenix PHOENIX-3534

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/phoenix/pull/303.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #303


commit a7712e3977830ffe715a2caaa577eb5cdb071e90
Author: rgidwani 
Date:   2017-03-17T19:21:56Z

Starting work on splittable System.Catalog

commit 220849fff34211a4b6356200365afe81d639cfc1
Author: rgidwani 
Date:   2017-03-20T20:44:54Z

Removed all references to multi-region System.Catalog for 
TableViewFinderResult

commit 356cd43e20be0d01c7ef5f97a9b9d26e213a140d
Author: rgidwani 
Date:   2017-03-31T23:11:59Z

Create table work, still trying to get rid of all columns

commit e01adb5f1c45d43a684371dcbf39eadbc381f9d2
Author: rgidwani 
Date:   2017-04-04T20:33:07Z

Create table and read views work now

commit 19c7ce54dcd1616ca8b8a7078fa8fc738e70f4f4
Author: rgidwani 
Date:   2017-04-04T20:35:30Z

Fixed the test - moving on to add drop columns

commit ab20f8da00145cc527c6f78dc7d493df7ac2f1b0
Author: rgidwani 
Date:   2017-04-04T22:59:10Z

getting tests and add column to work

commit ec3574453e8fae662a271f4340f82eaf90f52ce2
Author: rgidwani 
Date:   2017-04-05T23:02:29Z

Figuring out the delete logic and refactoring the old tests

commit 7d4133034b7167a7919eb9dd4ab19533ae9300ea
Author: rgidwani 
Date:   2017-04-11T22:25:32Z

Added proto timestamp and exluded values to pcolumn also took care of 
additive case where we take lower timestamp

commit adfc5ce7f9e7f55f801b5b8af5f414fd7ff96c23
Author: rgidwani 
Date:   2017-04-28T23:02:27Z

Drop Column Work in Progress, need to figure out how to resolve the combine 
logic for excluded columns

commit 24414bd942055769b68943d93e19013777ff7299
Author: rgidwani 
Date:   2017-05-01T22:32:35Z

Alter view drop column works!

commit 13b6e520e3f7a81bb257c7fe68bae81ededa6c99
Author: rgidwani 
Date:   2017-05-12T22:55:49Z

Drop Cascade and create check completed, need to test

commit 0313ceee0b0a7515323dc0fc402a6eac76786155
Author: rgidwani 
Date:   2017-05-15T20:40:14Z

Drop cascade seems to work

commit 590689f3243a388ae528850c3240427ff49a640c
Author: rgidwani 
Date:   2017-05-15T21:13:30Z

Phoenix 3534" to "PHOENIX-3534 Support multi region SYSTEM.CATALOG table

commit 96c0570bfcd767ad2342fb2fa22142eceed2b4eb
Author: rgidwani 
Date:   2017-05-24T19:32:05Z

Fixing up a few things, resolving read columns for child views doesn't 
always seem to work

commit ca64a0b36f5bf33c8ed3169096fce08ac768d695
Author: rgidwani 
Date:   2017-05-31T23:00:31Z

Adding in the child view constants

commit ac59c72784aff0478d9a481db5729827e0ba3cce
Author: rgidwani 
Date:   2017-05-31T23:00:31Z

Adding in the child view constants

commit 4d0ba3ac71812ed6bf79e383d7a7ab0ab8089287
Author: rgidwani 
Date:   2017-06-16T19:06:24Z

Merge branch 'splittableCatalog' into PHOENIX-3534

Conflicts:

phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java