You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by twdsilva <gi...@git.apache.org> on 2018/05/29 17:08:50 UTC

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

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 <rg...@...>
Date:   2017-03-17T19:21:56Z

    Starting work on splittable System.Catalog

commit 220849fff34211a4b6356200365afe81d639cfc1
Author: rgidwani <rg...@...>
Date:   2017-03-20T20:44:54Z

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

commit 356cd43e20be0d01c7ef5f97a9b9d26e213a140d
Author: rgidwani <rg...@...>
Date:   2017-03-31T23:11:59Z

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

commit e01adb5f1c45d43a684371dcbf39eadbc381f9d2
Author: rgidwani <rg...@...>
Date:   2017-04-04T20:33:07Z

    Create table and read views work now

commit 19c7ce54dcd1616ca8b8a7078fa8fc738e70f4f4
Author: rgidwani <rg...@...>
Date:   2017-04-04T20:35:30Z

    Fixed the test - moving on to add drop columns

commit ab20f8da00145cc527c6f78dc7d493df7ac2f1b0
Author: rgidwani <rg...@...>
Date:   2017-04-04T22:59:10Z

    getting tests and add column to work

commit ec3574453e8fae662a271f4340f82eaf90f52ce2
Author: rgidwani <rg...@...>
Date:   2017-04-05T23:02:29Z

    Figuring out the delete logic and refactoring the old tests

commit 7d4133034b7167a7919eb9dd4ab19533ae9300ea
Author: rgidwani <rg...@...>
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 <rg...@...>
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 <rg...@...>
Date:   2017-05-01T22:32:35Z

    Alter view drop column works!

commit 13b6e520e3f7a81bb257c7fe68bae81ededa6c99
Author: rgidwani <rg...@...>
Date:   2017-05-12T22:55:49Z

    Drop Cascade and create check completed, need to test

commit 0313ceee0b0a7515323dc0fc402a6eac76786155
Author: rgidwani <rg...@...>
Date:   2017-05-15T20:40:14Z

    Drop cascade seems to work

commit 590689f3243a388ae528850c3240427ff49a640c
Author: rgidwani <rg...@...>
Date:   2017-05-15T21:13:30Z

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

commit 96c0570bfcd767ad2342fb2fa22142eceed2b4eb
Author: rgidwani <rg...@...>
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 <rg...@...>
Date:   2017-05-31T23:00:31Z

    Adding in the child view constants

commit ac59c72784aff0478d9a481db5729827e0ba3cce
Author: rgidwani <rg...@...>
Date:   2017-05-31T23:00:31Z

    Adding in the child view constants

commit 4d0ba3ac71812ed6bf79e383d7a7ab0ab8089287
Author: rgidwani <rg...@...>
Date:   2017-06-16T19:06:24Z

    Merge branch 'splittableCatalog' into PHOENIX-3534
    
    Conflicts:
    	phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
    	phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
    	phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java

commit c2addd458dbb69542871ce10efc4a041a85d325d
Author: Thomas <td...@...>
Date:   2017-06-19T18:08:13Z

    test fixes

commit 91270753ea7c3197eed1694c47cf8691833677ba
Author: Thomas D'Silva <td...@...>
Date:   2017-09-14T21:55:46Z

    Merge branch 'master' of https://github.com/apache/phoenix into PHOENIX-3534

commit a4b64f01be6b0e21edab95753267666b3b4b7a39
Author: Thomas D'Silva <td...@...>
Date:   2017-09-15T03:13:36Z

    Remove unnessarch txn manager in ConnectionlessQueryServicesImpl

commit f1ea6621355152dcce236412cd5762e76097ffbc
Author: Thomas D'Silva <td...@...>
Date:   2017-09-15T03:15:07Z

    Merge branch 'master' into PHOENIX-3534

commit 571e88b1ee3da6e2bc64236e87e6e19130a588d0
Author: Thomas D'Silva <td...@...>
Date:   2017-10-05T23:15:00Z

    Rename link to VIEW_INDEX_PARENT

commit 308d2a89b819736122547febfddf96c7d98801ce
Author: Thomas D'Silva <td...@...>
Date:   2017-10-06T05:26:05Z

    Merge remote-tracking branch 'upstream/master' into PHOENIX-3534

commit f0e96705d3903c55b5449b09d15c4e3f128bf1ed
Author: Thomas D'Silva <td...@...>
Date:   2017-10-07T02:59:26Z

    fixed tests

commit 53fd995361a8a651247bd330ff1f36823c427f27
Author: Thomas D'Silva <td...@...>
Date:   2017-10-19T20:10:33Z

    add missing files

commit dcddc060bc789bd391302289659584c85baf2031
Author: Thomas D'Silva <td...@...>
Date:   2017-10-20T03:58:45Z

    add view index to parent links during upgrade

commit 508aa4bcf7e32d36e8f040bfb06b9de0ff8881d7
Author: Thomas D'Silva <td...@...>
Date:   2017-10-21T03:25:48Z

    fix tests

commit c32dfbbef5887885b9d003128d86bea9cb62327e
Author: Thomas D'Silva <td...@...>
Date:   2017-10-21T03:56:07Z

    fix test failures

commit abd98d92355c52cbc2c4264a70217ad2f46bb7d1
Author: Thomas D'Silva <td...@...>
Date:   2017-10-21T03:57:25Z

    Merge remote-tracking branch 'upstream/master' into PHOENIX-3534

commit 7c32d46dc721b3f0ab44e27521e8df569bead4d9
Author: Thomas D'Silva <td...@...>
Date:   2017-10-26T22:18:13Z

    minor

----


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<Mutation> 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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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<PTable, MetaDataProtos.MutationCode> 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<PTable, MetaDataProtos.MutationCode>(table,
    +                    MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
    +        }
    +        if (!skipAddingParentColumns) {
    +            table =
    +                    addDerivedColumnsFromAncestors(table, timestamp, clientVersion,
    +                        lockedAncestorTable);
    +            if (table==null) {
    +                return new Pair<PTable, MetaDataProtos.MutationCode>(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<PTable> 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<PTable, MetaDataProtos.MutationCode>(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()
    +                        : ByteUtil.EMPTY_BYTE_ARRAY;
    +        byte[] schemaName = table.getSchemaName().getBytes();
    +        byte[] tableName = table.getTableName().getBytes();
    +		String fullTableName = SchemaUtil.getTableName(table.getSchemaName().getString(),
    +				table.getTableName().getString());
    +        boolean hasIndexId = table.getViewIndexId() != null;
    +        boolean isSalted = table.getBucketNum() != null;
    +        if (table.getType() != PTableType.VIEW && !hasIndexId) {
    +            return table;
    +        }
    +        boolean isDiverged = isDivergedView(table);
    +        // here you combine columns from the parent tables the logic is as follows, if the PColumn
    +        // is in the EXCLUDED_COLUMNS remove it, otherwise priority of keeping duplicate columns is
    +        // child -> parent
    +        List<TableInfo> ancestorList = Lists.newArrayList();
    +        TableViewFinderResult viewFinderResult = new TableViewFinderResult();
    +        if (PTableType.VIEW == table.getType()) {
    +            findAncestorViews(tenantId, schemaName, tableName, viewFinderResult,
    +                table.isNamespaceMapped());
    +        } else { // is a view index
    +            findAncestorViewsOfIndex(tenantId, schemaName, tableName, viewFinderResult,
    +                table.isNamespaceMapped());
    +        }
    +        if (viewFinderResult.getResults().isEmpty()) {
    +            // no need to combine columns for local indexes on regular tables
    +            return table;
    +        }
    +        for (TableInfo viewInfo : viewFinderResult.getResults()) {
    +            ancestorList.add(viewInfo);
    +        }
    +        List<PColumn> allColumns = Lists.newArrayList();
    +        List<PColumn> excludedColumns = Lists.newArrayList();
    +        // add my own columns first in reverse order
    +        List<PColumn> myColumns = table.getColumns();
    +        for (int i = myColumns.size() - 1; i >= 0; i--) {
    +            PColumn pColumn = myColumns.get(i);
    +            if (pColumn.isExcluded()) {
    +                excludedColumns.add(pColumn);
    +            } else if (!pColumn.equals(SaltingUtil.SALTING_COLUMN)) { 
    +                // skip salted column as it will be added from the base table columns
    +                allColumns.add(pColumn);
    +            }
    +        }
    +
    +        // initialize map from with indexed expression to list of required data columns
    +        // then remove the data columns that have not been dropped, so that we get the columns that
    +        // have been dropped
    +        Map<PColumn, List<String>> indexRequiredDroppedDataColMap =
    +                Maps.newHashMapWithExpectedSize(table.getColumns().size());
    +        if (hasIndexId) {
    +            int indexPosOffset = (isSalted ? 1 : 0) + (table.isMultiTenant() ? 1 : 0) + 1;
    +            ColumnNameTrackingExpressionCompiler expressionCompiler =
    +                    new ColumnNameTrackingExpressionCompiler();
    +            for (int i = indexPosOffset; i < table.getPKColumns().size(); i++) {
    +                PColumn indexColumn = table.getPKColumns().get(i);
    +                try {
    +                    expressionCompiler.reset();
    +                    String expressionStr = IndexUtil.getIndexColumnExpressionStr(indexColumn);
    +                    ParseNode parseNode = SQLParser.parseCondition(expressionStr);
    +                    parseNode.accept(expressionCompiler);
    +                    indexRequiredDroppedDataColMap.put(indexColumn,
    +                        Lists.newArrayList(expressionCompiler.getDataColumnNames()));
    +                } catch (SQLException e) {
    +                    throw new RuntimeException(e); // Impossible
    +                }
    +            }
    +        }
    +
    +        // now go up from child to parent all the way to the base table:
    +        PTable baseTable = null;
    +        long maxTableTimestamp = -1;
    +        int numPKCols = table.getPKColumns().size();
    +        for (int i = 0; i < ancestorList.size(); i++) {
    +            TableInfo parentTableInfo = ancestorList.get(i);
    +            PTable pTable = null;
    +            String fullParentTableName = SchemaUtil.getTableName(parentTableInfo.getSchemaName(),
    +			    parentTableInfo.getTableName());
    +            PName parentTenantId =
    +                    (parentTableInfo.getTenantId() != null && parentTableInfo.getTenantId().length!=0)
    +                            ? PNameFactory.newName(parentTableInfo.getTenantId()) : null;
    +            PTableKey pTableKey = new PTableKey(parentTenantId, fullParentTableName);
    +            // if we already have the PTable of an ancestor that has been locked, no need to look up
    +            // the table
    +            if (lockedAncestorTable != null && lockedAncestorTable.getKey().equals(pTableKey)) {
    +                pTable = lockedAncestorTable;
    +            } else {
    +                // if we are currently combining columns for a view index and are looking up its
    +                // ancestors we do not add the indexes to the ancestor PTable (or else we end up in
    +                // a circular loop)
    +                // we also don't need to add parent columns of the ancestors as we combine columns
    +                // from all ancestors
    +                pTable =
    +                        doGetTable(parentTableInfo.getTenantId(), parentTableInfo.getSchemaName(),
    +                            parentTableInfo.getTableName(), timestamp, null, clientVersion, hasIndexId,
    +                            true, null);
    +            }
    +            if (pTable == null) {
    +                throw new ParentTableNotFoundException(parentTableInfo, fullTableName);
    +            } else {
    +                // only combine columns for view indexes (and not local indexes on regular tables
    +                // which also have a viewIndexId)
    +                if (i == 0 && hasIndexId && pTable.getType() != PTableType.VIEW) {
    +                    return table;
    +                }
    +                if (TABLE.equals(pTable.getType())) {
    +                    baseTable = pTable;
    +                }
    +                // set the final table timestamp as the max timestamp of the view/view index or its
    +                // ancestors
    +                maxTableTimestamp = Math.max(maxTableTimestamp, pTable.getTimeStamp());
    +                if (hasIndexId) {
    +                    // add all pk columns of parent tables to indexes
    +                    for (PColumn column : pTable.getPKColumns()) {
    +                        // don't add the salt column of ancestor tables for view indexes
    +                        if (column.equals(SaltingUtil.SALTING_COLUMN) || column.isExcluded()) {
    --- End diff --
    
    Same comment as before - we should be able to match based on the column being the first column. Note that the salt column would only be in the base physical table.


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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<MetaDataResponse> 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<Mutation> 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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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<ImmutableBytesPtr> 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<Mutation> 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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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<PColumn> 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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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<PTable, MetaDataProtos.MutationCode> 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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<String> 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<String> getDataColumnNames() {
    +		return dataColumnNames;
    +	}
    +
    +}
    --- End diff --
    
    Fixed.


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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.
    -        if (!isMultiTenant) {
    -            byte[] startRow = ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY);
    -            byte[] stopRow = ByteUtil.nextKey(startRow);
    -            scan.setStartRow(startRow);
    -            scan.setStopRow(stopRow);
    -        }
    -        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, linkTypeBytes);
    -        SingleColumnValueFilter tableTypeFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES,
    -                CompareOp.EQUAL, PTableType.VIEW.getSerializedValue().getBytes());
    -        tableTypeFilter.setFilterIfMissing(false);
    -        linkFilter.setFilterIfMissing(true);
    -        byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil
    -                .getPhysicalHBaseTableName(schemaName, tableName, table.isNamespaceMapped())
    -                .getBytes());
    -        SuffixFilter rowFilter = new SuffixFilter(suffix);
    -        List<Filter> filters = Lists.<Filter>newArrayList(linkFilter,tableTypeFilter,rowFilter);
    -        if (stopAfterFirst) {
    -            filters.add(new PageFilter(1));
    -        }
    -        FilterList filter = new FilterList(filters);
    -        scan.setFilter(filter);
    -        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
    -        
    -        // Original region-only scanner modified due to PHOENIX-1208
    -        // RegionScanner scanner = region.getScanner(scan);
    -        // The following *should* work, but doesn't due to HBASE-11837
    -        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
    -        // HTableInterface hTable = env.getTable(systemCatalogTableName);
    -        // These deprecated calls work around the issue
    -        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
    -            region.getTableDesc().getTableName().getName())) {
    -            boolean allViewsInCurrentRegion = true;
    -            int numOfChildViews = 0;
    -            List<ViewInfo> viewInfoList = Lists.newArrayList();
    -            try (ResultScanner scanner = hTable.getScanner(scan)) {
    -                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
    -                    numOfChildViews++;
    -                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    -                    ResultTuple resultTuple = new ResultTuple(result);
    -                    resultTuple.getKey(ptr);
    -                    byte[] key = ptr.copyBytes();
    -                    if (checkTableKeyInRegion(key, region) != null) {
    -                        allViewsInCurrentRegion = false;
    -                    }
    -                    byte[][] rowKeyMetaData = new byte[3][];
    -                    getVarChars(result.getRow(), 3, rowKeyMetaData);
    -                    byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    -                    byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    -                    byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    -                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
    -                }
    -                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
    -                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
    -                    tableViewFinderResult.setAllViewsNotInSingleRegion();
    -                }
    -                return tableViewFinderResult;
    +
    +    private void findAncestorViewsOfIndex(byte[] tenantId, byte[] schemaName, byte[] indexName,
    +            TableViewFinderResult result, boolean isNamespaceMapped) throws IOException {
    +        try (Table hTable =
    +                env.getTable(SchemaUtil.getPhysicalTableName(
    +                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
    +            TableViewFinderResult currentResult =
    +                    ViewFinder.findParentViewofIndex(hTable, tenantId, schemaName, indexName);
    +            if (currentResult.getResults().size() == 1) {
    +                result.addResult(currentResult);
    +                TableInfo tableInfo = currentResult.getResults().get(0);
    +                findAncestorViews(tableInfo.getTenantId(), tableInfo.getSchemaName(),
    +                    tableInfo.getTableName(), result, isNamespaceMapped);
                 }
    +            // else this is an index on a regular table and so we don't need to combine columns
             }
         }
         
    -    private TableViewFinder findChildViews_4_11(Region region, byte[] tenantId, byte[] schemaName, byte[] tableName, boolean stopAfterFirst) throws IOException {
    -        Scan scan = new Scan();
    -        byte[] startRow = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    -        byte[] stopRow = ByteUtil.nextKey(startRow);
    -        scan.setStartRow(startRow);
    -        scan.setStopRow(stopRow);
    -        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, CHILD_TABLE_BYTES);
    -        Filter filter = linkFilter;
    -        linkFilter.setFilterIfMissing(true);
    -        if (stopAfterFirst) {
    -            filter = new FilterList(linkFilter, new PageFilter(1));
    -        }
    -        scan.setFilter(filter);
    -        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
    -        
    -        // Original region-only scanner modified due to PHOENIX-1208
    -        // RegionScanner scanner = region.getScanner(scan);
    -        // The following *should* work, but doesn't due to HBASE-11837
    -        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
    -        // HTableInterface hTable = env.getTable(systemCatalogTableName);
    -        // These deprecated calls work around the issue
    -        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
    -            region.getTableDesc().getTableName().getName())) {
    -            boolean allViewsInCurrentRegion = true;
    -            int numOfChildViews = 0;
    -            List<ViewInfo> viewInfoList = Lists.newArrayList();
    -            try (ResultScanner scanner = hTable.getScanner(scan)) {
    -                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
    -                    numOfChildViews++;
    -                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    -                    ResultTuple resultTuple = new ResultTuple(result);
    -                    resultTuple.getKey(ptr);
    -                    byte[] key = ptr.copyBytes();
    -                    if (checkTableKeyInRegion(key, region) != null) {
    -                        allViewsInCurrentRegion = false;
    -                    }
    -                    byte[][] rowViewKeyMetaData = new byte[5][];
    -                    getVarChars(result.getRow(), 5, rowViewKeyMetaData);
    -                    byte[] viewTenantId = rowViewKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
    -                    byte[] viewSchemaName = SchemaUtil.getSchemaNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
    -                    byte[] viewName = SchemaUtil.getTableNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
    -                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
    -                }
    -                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
    -                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
    -                    tableViewFinderResult.setAllViewsNotInSingleRegion();
    -                }
    -                return tableViewFinderResult;
    +    private void findAncestorViews(byte[] tenantId, byte[] schemaName, byte[] tableName,
    +            TableViewFinderResult result, boolean isNamespaceMapped) throws IOException {
    +        try (Table hTable =
    +                env.getTable(SchemaUtil.getPhysicalTableName(
    +                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
    +            ViewFinder.findAllRelatives(hTable, tenantId, schemaName, tableName,
    +                LinkType.PARENT_TABLE, result);
    +            if (!isNamespaceMapped || schemaName.length==0) {
    +                // the child->parent link is overwritten by the child->physical table link for first
    +                // level children of base table when namespace mapping is disabled or if the parent
    +                // table doesn't have a schema as both the parent table name and physical table name
    +                // are the same.
    +                // When namespace mapping is enabled the physical table name is of the form S:T
    +                // while the table name is of the form S.T so we need to query for the
    +                // PHYSICAL_TABLE link
    --- End diff --
    
    Since the linking rows are being re-written I believe in 4.15, should we change this behavior?


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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 issue #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG table

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on the issue:

    https://github.com/apache/phoenix/pull/303
  
    @JamesRTaylor  Thanks for the feedback, I have updated the PR. I will get this committed shortly.


---

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

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/303#discussion_r201745438
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -3642,30 +3596,62 @@ private void dropIndexes(PTable table, Region region, List<ImmutableBytesPtr> 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<Mutation> 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 --
    
    This is to handle dropping indexes on views. We could drop a column on a base table which is used by an index on view which could be on a separate region.


---

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

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<MetaDataResponse> 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<Mutation> 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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(COLUMN_DEF), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SQL_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATA_TYPE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SQL_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SQL_DATETIME_SUB_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATETIME_SUB),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SQL_DATETIME_SUB), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl CHAR_OCTET_LENGTH_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, Bytes.toBytes(CHAR_OCTET_LENGTH), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl ORDINAL_POSITION_COLUMN = new PColumnImpl(PNameFactory.newName(ORDINAL_POSITION),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, ORDINAL_POSITION_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl IS_NULLABLE_COLUMN = new PColumnImpl(PNameFactory.newName(IS_NULLABLE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(IS_NULLABLE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_CATALOG_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_CATALOG),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_SCHEMA_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_SCHEMA),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_SCHEMA), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_TABLE_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_TABLE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_TABLE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SOURCE_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SOURCE_DATA_TYPE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SOURCE_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl IS_AUTOINCREMENT_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl ARRAY_SIZE_COLUMN = new PColumnImpl(PNameFactory.newName(ARRAY_SIZE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, ARRAY_SIZE_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl COLUMN_FAMILY_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_FAMILY),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, COLUMN_FAMILY_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl TYPE_ID_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, Bytes.toBytes(TYPE_ID), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl VIEW_CONSTANT_COLUMN = new PColumnImpl(PNameFactory.newName(VIEW_CONSTANT),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarbinary.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, VIEW_CONSTANT_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl MULTI_TENANT_COLUMN = new PColumnImpl(PNameFactory.newName(MULTI_TENANT),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PBoolean.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, MULTI_TENANT_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl KEY_SEQ_COLUMN = new PColumnImpl(PNameFactory.newName(KEY_SEQ),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, KEY_SEQ_BYTES, HConstants.LATEST_TIMESTAMP);
    +	
    +	private static final List<PColumnImpl> PK_DATUM_LIST = Lists.newArrayList(TENANT_ID_COLUMN, TABLE_SCHEM_COLUMN, TABLE_NAME_COLUMN, COLUMN_NAME_COLUMN);
    +	
    +	private static final RowProjector GET_COLUMNS_ROW_PROJECTOR = new RowProjector(
    +			Arrays.<ColumnProjector> asList(
    +					new ExpressionProjector(TABLE_CAT, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TENANT_ID_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 0)), false),
    +					new ExpressionProjector(TABLE_SCHEM, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TABLE_SCHEM_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 1)), false),
    +					new ExpressionProjector(TABLE_NAME, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TABLE_NAME_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 2)), false),
    +					new ExpressionProjector(COLUMN_NAME, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(COLUMN_NAME_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 3)), false),
    +					new ExpressionProjector(DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(TYPE_NAME, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(TYPE_NAME_COLUMN), false),
    +					new ExpressionProjector(COLUMN_SIZE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_SIZE_COLUMN), false),
    +					new ExpressionProjector(BUFFER_LENGTH, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(BUFFER_LENGTH_COLUMN), false),
    +					new ExpressionProjector(DECIMAL_DIGITS, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(DECIMAL_DIGITS_COLUMN), false),
    +					new ExpressionProjector(NUM_PREC_RADIX, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(NUM_PREC_RADIX_COLUMN), false),
    +					new ExpressionProjector(NULLABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(NULLABLE_COLUMN), false),
    +					new ExpressionProjector(REMARKS, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(REMARKS_COLUMN), false),
    +					new ExpressionProjector(COLUMN_DEF, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_DEF_COLUMN), false),
    +					new ExpressionProjector(SQL_DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SQL_DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(SQL_DATETIME_SUB, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SQL_DATETIME_SUB_COLUMN), false),
    +					new ExpressionProjector(CHAR_OCTET_LENGTH, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(CHAR_OCTET_LENGTH_COLUMN), false),
    +					new ExpressionProjector(ORDINAL_POSITION, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(ORDINAL_POSITION_COLUMN), false),
    +					new ExpressionProjector(IS_NULLABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(IS_NULLABLE_COLUMN), false),
    +					new ExpressionProjector(SCOPE_CATALOG, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_CATALOG_COLUMN), false),
    +					new ExpressionProjector(SCOPE_SCHEMA, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_SCHEMA_COLUMN), false),
    +					new ExpressionProjector(SCOPE_TABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_TABLE_COLUMN), false),
    +					new ExpressionProjector(SOURCE_DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SOURCE_DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(IS_AUTOINCREMENT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(IS_AUTOINCREMENT_COLUMN), false),
    +					new ExpressionProjector(ARRAY_SIZE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(ARRAY_SIZE_COLUMN), false),
    +					new ExpressionProjector(COLUMN_FAMILY, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_FAMILY_COLUMN), false),
    +					new ExpressionProjector(TYPE_ID, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(TYPE_ID_COLUMN), false),
    +					new ExpressionProjector(VIEW_CONSTANT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(VIEW_CONSTANT_COLUMN), false),
    +					new ExpressionProjector(MULTI_TENANT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(MULTI_TENANT_COLUMN), false),
    +					new ExpressionProjector(KEY_SEQ, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(KEY_SEQ_COLUMN), false)
    +					), 0, true);
    +	
    +	private boolean match(String str, String pattern) throws SQLException {
    +		LiteralExpression strExpr = LiteralExpression.newConstant(str, PVarchar.INSTANCE, SortOrder.ASC);
    +		LiteralExpression patternExpr = LiteralExpression.newConstant(pattern, PVarchar.INSTANCE, SortOrder.ASC);
    +		List<Expression> children = Arrays.<Expression>asList(strExpr, patternExpr);
    +		LikeExpression likeExpr = StringBasedLikeExpression.create(children, LikeType.CASE_SENSITIVE);
    +		ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    +		boolean evaluated = likeExpr.evaluate(null, ptr);
    +		Boolean result = (Boolean)likeExpr.getDataType().toObject(ptr);
    +		if (evaluated) {
    +			return result;
    +		}
    +		return false;
    +	}
    +    
    +	@Override
         public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
                 throws SQLException {
    -        StringBuilder buf = new StringBuilder("select \n " +
    -                TENANT_ID + " " + TABLE_CAT + "," + // use this for tenant id
    -                TABLE_SCHEM + "," +
    -                TABLE_NAME + " ," +
    -                COLUMN_NAME + "," +
    -                ExternalSqlTypeIdFunction.NAME + "(" + DATA_TYPE + ") AS " + DATA_TYPE + "," +
    -                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + "," +
    -                COLUMN_SIZE + "," +
    -                BUFFER_LENGTH + "," +
    -                DECIMAL_DIGITS + "," +
    -                NUM_PREC_RADIX + "," +
    -                NULLABLE + "," +
    -                REMARKS + "," +
    -                COLUMN_DEF + "," +
    -                SQL_DATA_TYPE + "," +
    -                SQL_DATETIME_SUB + "," +
    -                CHAR_OCTET_LENGTH + "," +
    -                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + ORDINAL_POSITION + "-1 ELSE " + ORDINAL_POSITION + " END AS " + ORDINAL_POSITION + "," +
    -                "CASE " + NULLABLE + " WHEN " + DatabaseMetaData.attributeNoNulls +  " THEN '" + Boolean.FALSE.toString() + "' WHEN " + DatabaseMetaData.attributeNullable + " THEN '" + Boolean.TRUE.toString() + "' END AS " + IS_NULLABLE + "," +
    -                SCOPE_CATALOG + "," +
    -                SCOPE_SCHEMA + "," +
    -                SCOPE_TABLE + "," +
    -                SOURCE_DATA_TYPE + "," +
    -                IS_AUTOINCREMENT + "," +
    -                ARRAY_SIZE + "," +
    -                COLUMN_FAMILY + "," +
    -                DATA_TYPE + " " + TYPE_ID + "," +// raw type id for potential internal consumption
    -                VIEW_CONSTANT + "," +
    -                MULTI_TENANT + "," +
    -                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + KEY_SEQ + "-1 ELSE " + KEY_SEQ + " END AS " + KEY_SEQ +
    -                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS + "(" + TENANT_POS_SHIFT + " BOOLEAN)");
    -        StringBuilder where = new StringBuilder();
    -        addTenantIdFilter(where, catalog);
    -        if (schemaPattern != null) {
    -            appendConjunction(where);
    -            where.append(TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'" ));
    -        }
    -        if (tableNamePattern != null && tableNamePattern.length() > 0) {
    -            appendConjunction(where);
    -            where.append(TABLE_NAME + " like '" + StringUtil.escapeStringConstant(tableNamePattern) + "'" );
    -        }
    -        // Allow a "." in columnNamePattern for column family match
    -        String colPattern = null;
    -        if (columnNamePattern != null && columnNamePattern.length() > 0) {
    +		boolean isTenantSpecificConnection = connection.getTenantId() != null;
    +    	List<Tuple> tuples = Lists.newArrayListWithExpectedSize(10);
    +    	ResultSet rs = getTables(catalog, schemaPattern, tableNamePattern, null);
    +    	while (rs.next()) {
    +    		String schemaName = rs.getString(TABLE_SCHEM);
    +    		boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
    +    		// Allow a "." in columnNamePattern for column family match
    +            String colPattern = null;
                 String cfPattern = null;
    -            int index = columnNamePattern.indexOf('.');
    -            if (index <= 0) {
    -                colPattern = columnNamePattern;
    -            } else {
    -                cfPattern = columnNamePattern.substring(0, index);
    -                if (columnNamePattern.length() > index+1) {
    -                    colPattern = columnNamePattern.substring(index+1);
    +            if (columnNamePattern != null && columnNamePattern.length() > 0) {
    +                int index = columnNamePattern.indexOf('.');
    +                if (index <= 0) {
    +                    colPattern = columnNamePattern;
    +                } else {
    +                    cfPattern = columnNamePattern.substring(0, index);
    +                    if (columnNamePattern.length() > index+1) {
    +                        colPattern = columnNamePattern.substring(index+1);
    +                    }
                     }
                 }
    -            if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
    -                // Will pick up only KV columns
    -                appendConjunction(where);
    -                where.append(COLUMN_FAMILY + " like '" + StringUtil.escapeStringConstant(cfPattern) + "'" );
    -            }
    -            if (colPattern != null && colPattern.length() > 0) {
    -                appendConjunction(where);
    -                where.append(COLUMN_NAME + " like '" + StringUtil.escapeStringConstant(colPattern) + "'" );
    -            }
    -        }
    -        if (colPattern == null || colPattern.length() == 0) {
    -            appendConjunction(where);
    -            where.append(COLUMN_NAME + " is not null" );
    -            appendConjunction(where);
    -            where.append(LINK_TYPE + " is null" );
    -        }
    -        boolean isTenantSpecificConnection = connection.getTenantId() != null;
    -        if (isTenantSpecificConnection) {
    -            buf.append(" where (" + where + ") OR ("
    -                    + COLUMN_FAMILY + " is null AND " +  COLUMN_NAME + " is null)");
    -        } else {
    -            buf.append(" where " + where);
    -        }
    -        buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME + "," + SYSTEM_CATALOG_ALIAS + "." + ORDINAL_POSITION);
    -
    -        Statement stmt;
    -        if (isTenantSpecificConnection) {
    -            stmt = connection.createStatement(new PhoenixStatementFactory() {
    -                @Override
    -                public PhoenixStatement newStatement(PhoenixConnection connection) {
    -                    return new PhoenixStatement(connection) {
    -                        @Override
    -                        public PhoenixResultSet newResultSet(ResultIterator iterator, RowProjector projector,
    -                                StatementContext context) throws SQLException {
    -                            return new PhoenixResultSet(new TenantColumnFilteringIterator(iterator, projector),
    -                                    projector, context);
    -                        }
    -                    };
    +			String tableName = rs.getString(TABLE_NAME);
    +    		String tenantId = rs.getString(TABLE_CAT);
    +    		String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
    +    		// if its a SYSTEM table we prevent the rpc to fetch the table so look it up in the cache
    +			PTable table = systemTable ? PhoenixRuntime.getTable(connection, fullTableName)
    +					: PhoenixRuntime.getTableNoCache(connection, fullTableName);
    +    		boolean isSalted = table.getBucketNum()!=null;
    +    		boolean tenantColSkipped = false;
    +    		for (PColumn column : table.getColumns()) {
    +    			if (isTenantSpecificConnection && column.equals(table.getPKColumns().get(0))) {
    +    				// skip the tenant column
    +    				tenantColSkipped = true;
    +    				continue;
    +    			}
    +    			String columnFamily = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
    +    			String columnName = column.getName().getString();
    +    			if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
    +    				if (columnFamily==null || !match(columnFamily, cfPattern)) {
    +    					continue;
    +    				}
                     }
    -            });
    -        } else {
    -            stmt = connection.createStatement();
    -        }
    -        return stmt.executeQuery(buf.toString());
    -    }
    -    
    -//    private ColumnResolver getColumnResolverForCatalogTable() throws SQLException {
    -//        TableRef tableRef = new TableRef(getTable(connection, SYSTEM_CATALOG_NAME));
    -//        return FromCompiler.getResolver(tableRef);
    -//    }
    -    
    -    /**
    -     * Filters the tenant id column out of a column metadata result set (thus, where each row is a column definition).
    -     * The tenant id is by definition the first column of the primary key, but the primary key does not necessarily
    -     * start at the first column. Assumes columns are sorted on ordinal position.
    -     */
    -    private static class TenantColumnFilteringIterator extends DelegateResultIterator {
    -        private final RowProjector rowProjector;
    -        private final int columnFamilyIndex;
    -        private final int columnNameIndex;
    -        private final int multiTenantIndex;
    -        private final int keySeqIndex;
    -        private boolean inMultiTenantTable;
    -        private boolean tenantColumnSkipped;
    -
    -        private TenantColumnFilteringIterator(ResultIterator delegate, RowProjector rowProjector) throws SQLException {
    -            super(delegate);
    -            this.rowProjector = rowProjector;
    -            this.columnFamilyIndex = rowProjector.getColumnIndex(COLUMN_FAMILY);
    -            this.columnNameIndex = rowProjector.getColumnIndex(COLUMN_NAME);
    -            this.multiTenantIndex = rowProjector.getColumnIndex(MULTI_TENANT);
    -            this.keySeqIndex = rowProjector.getColumnIndex(KEY_SEQ);
    -        }
    -
    -        @Override
    -        public Tuple next() throws SQLException {
    -            Tuple tuple = super.next();
    -
    -            while (tuple != null
    -                    && getColumn(tuple, columnFamilyIndex) == null && getColumn(tuple, columnNameIndex) == null) {
    -                // new table, check if it is multitenant
    -                inMultiTenantTable = getColumn(tuple, multiTenantIndex) == Boolean.TRUE;
    -                tenantColumnSkipped = false;
    -                // skip row representing table
    -                tuple = super.next();
    -            }
    -
    -            if (tuple != null && inMultiTenantTable && !tenantColumnSkipped) {
    -                Object value = getColumn(tuple, keySeqIndex);
    -                if (value != null && ((Number)value).longValue() == 1L) {
    -                    tenantColumnSkipped = true;
    -                    // skip tenant id primary key column
    -                    return next();
    +                if (colPattern != null && colPattern.length() > 0) {
    +                	if (!match(columnName, colPattern)) {
    +                		continue;
    +                	}
                     }
    -            }
    -
    -            if (tuple != null && tenantColumnSkipped) {
    -                ResultTuple resultTuple = (ResultTuple)tuple;
    -                List<Cell> cells = resultTuple.getResult().listCells();
    -                KeyValue kv = new KeyValue(resultTuple.getResult().getRow(), TABLE_FAMILY_BYTES,
    -                        TENANT_POS_SHIFT_BYTES, PDataType.TRUE_BYTES);
    -                List<Cell> newCells = Lists.newArrayListWithCapacity(cells.size() + 1);
    -                newCells.addAll(cells);
    -                newCells.add(kv);
    -                Collections.sort(newCells, KeyValue.COMPARATOR);
    -                tuple = new ResultTuple(Result.create(newCells));
    -            }
    -            return tuple;
    -        }
    -
    -        private Object getColumn(Tuple tuple, int index) throws SQLException {
    -            ColumnProjector projector = this.rowProjector.getColumnProjector(index);
    -            PDataType type = projector.getExpression().getDataType();
    -            return projector.getValue(tuple, type, new ImmutableBytesPtr());
    -        }
    +                // generate row key
    +                // TENANT_ID, TABLE_SCHEM, TABLE_NAME , COLUMN_NAME are row key columns
    +                byte[] rowKey =
    +                        SchemaUtil.getColumnKey(tenantId, schemaName, tableName, columnName, null);
    +
    +                // add one cell for each column info
    +                List<Cell> cells = Lists.newArrayListWithCapacity(25);
    +                // DATA_TYPE
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, DATA_TYPE_BYTES,
    +                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    PInteger.INSTANCE.toBytes(column.getDataType().getResultSetSqlType())));
    +                // TYPE_NAME
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
    +                    Bytes.toBytes(TYPE_NAME), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    column.getDataType().getSqlTypeNameBytes()));
    +                // COLUMN_SIZE
    +                cells.add(
    +                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, COLUMN_SIZE_BYTES,
    +                        MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                        column.getMaxLength() != null
    +                                ? PInteger.INSTANCE.toBytes(column.getMaxLength())
    +                                : ByteUtil.EMPTY_BYTE_ARRAY));
    +                // BUFFER_LENGTH
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
    +                    Bytes.toBytes(BUFFER_LENGTH), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    ByteUtil.EMPTY_BYTE_ARRAY));
    +                // DECIMAL_DIGITS
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES,
    +                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    column.getScale() != null ? PInteger.INSTANCE.toBytes(column.getScale())
    +                            : ByteUtil.EMPTY_BYTE_ARRAY));
    +                // NUM_PREC_RADIX
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
    +                    Bytes.toBytes(NUM_PREC_RADIX), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    ByteUtil.EMPTY_BYTE_ARRAY));
    +                // NULLABLE
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, NULLABLE_BYTES,
    +                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    PInteger.INSTANCE.toBytes(SchemaUtil.getIsNullableInt(column.isNullable()))));
    +                // REMARKS
    +                cells.add(
    +                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, Bytes.toBytes(REMARKS),
    +                        MetaDataProtocol.MIN_TABLE_TIMESTAMP, ByteUtil.EMPTY_BYTE_ARRAY));
    --- End diff --
    
    Is it required to add a cell for REMARKS since it will never have a value?


---

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

Posted by twdsilva <gi...@git.apache.org>.
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<Mutation> tableMetadata = ProtobufUtil.getMutations(request);
    +            List<Mutation> 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<byte[]> tableNamesToDelete = Lists.newArrayList();
                 List<SharedTableState> 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 ...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/303#discussion_r201776089
  
    --- 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<MetaDataResponse> 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<Mutation> 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 --
    
    Done.


---

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

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by churrodog <gi...@git.apache.org>.
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<PColumn> 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));
    +				viewColumnConstantsMatched[column.getPosition()]=true;
    +            }
    +        	// If view is not updatable, viewColumnConstants should be empty. We will still
    +            // inherit our parent viewConstants, but we have no additional ones.
    +        	else if(isViewReferenced ){
    +        		result.add(new PColumnImpl(column, column.getViewConstant(), isViewReferenced));
    +        	}
    +        	else {
    +                result.add(column);
    +            }
    +        }
    +		// ensure that node of the columns in the view where statement were
    +		// dropped in any of this views ancestors
    +//		for (int i = 0; i < viewColumnConstantsMatched.length; ++i) {
    --- End diff --
    
    should this be removed, or is this still valid?


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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.
    -        if (!isMultiTenant) {
    -            byte[] startRow = ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY);
    -            byte[] stopRow = ByteUtil.nextKey(startRow);
    -            scan.setStartRow(startRow);
    -            scan.setStopRow(stopRow);
    -        }
    -        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, linkTypeBytes);
    -        SingleColumnValueFilter tableTypeFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES,
    -                CompareOp.EQUAL, PTableType.VIEW.getSerializedValue().getBytes());
    -        tableTypeFilter.setFilterIfMissing(false);
    -        linkFilter.setFilterIfMissing(true);
    -        byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil
    -                .getPhysicalHBaseTableName(schemaName, tableName, table.isNamespaceMapped())
    -                .getBytes());
    -        SuffixFilter rowFilter = new SuffixFilter(suffix);
    -        List<Filter> filters = Lists.<Filter>newArrayList(linkFilter,tableTypeFilter,rowFilter);
    -        if (stopAfterFirst) {
    -            filters.add(new PageFilter(1));
    -        }
    -        FilterList filter = new FilterList(filters);
    -        scan.setFilter(filter);
    -        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
    -        
    -        // Original region-only scanner modified due to PHOENIX-1208
    -        // RegionScanner scanner = region.getScanner(scan);
    -        // The following *should* work, but doesn't due to HBASE-11837
    -        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
    -        // HTableInterface hTable = env.getTable(systemCatalogTableName);
    -        // These deprecated calls work around the issue
    -        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
    -            region.getTableDesc().getTableName().getName())) {
    -            boolean allViewsInCurrentRegion = true;
    -            int numOfChildViews = 0;
    -            List<ViewInfo> viewInfoList = Lists.newArrayList();
    -            try (ResultScanner scanner = hTable.getScanner(scan)) {
    -                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
    -                    numOfChildViews++;
    -                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    -                    ResultTuple resultTuple = new ResultTuple(result);
    -                    resultTuple.getKey(ptr);
    -                    byte[] key = ptr.copyBytes();
    -                    if (checkTableKeyInRegion(key, region) != null) {
    -                        allViewsInCurrentRegion = false;
    -                    }
    -                    byte[][] rowKeyMetaData = new byte[3][];
    -                    getVarChars(result.getRow(), 3, rowKeyMetaData);
    -                    byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    -                    byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    -                    byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    -                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
    -                }
    -                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
    -                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
    -                    tableViewFinderResult.setAllViewsNotInSingleRegion();
    -                }
    -                return tableViewFinderResult;
    +
    +    private void findAncestorViewsOfIndex(byte[] tenantId, byte[] schemaName, byte[] indexName,
    --- End diff --
    
    Is this running a new scan to find the ancestor links? A comment here might be good.


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 issue #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG table

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on the issue:

    https://github.com/apache/phoenix/pull/303
  
    +1 to the patch. Great work @twdsilva and @churrodog! I made some minor comments for some potential follow up work and had a few questions, but let's get this committed first. I'd recommend the following priority for the next JIRA as:
    
    1. Move views to their own table
    2. Get rid of client side code that is sending the base columns
    3. Fix corner case/race condition issues
    4. Add code that doesn't write orphaned metadata on major compaction


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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<PTable, MetaDataProtos.MutationCode> 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<PTable, MetaDataProtos.MutationCode>(table,
    +                    MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
    +        }
    +        if (!skipAddingParentColumns) {
    +            table =
    +                    addDerivedColumnsFromAncestors(table, timestamp, clientVersion,
    +                        lockedAncestorTable);
    +            if (table==null) {
    +                return new Pair<PTable, MetaDataProtos.MutationCode>(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<PTable> 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<PTable, MetaDataProtos.MutationCode>(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()
    +                        : ByteUtil.EMPTY_BYTE_ARRAY;
    +        byte[] schemaName = table.getSchemaName().getBytes();
    +        byte[] tableName = table.getTableName().getBytes();
    +		String fullTableName = SchemaUtil.getTableName(table.getSchemaName().getString(),
    +				table.getTableName().getString());
    +        boolean hasIndexId = table.getViewIndexId() != null;
    +        boolean isSalted = table.getBucketNum() != null;
    +        if (table.getType() != PTableType.VIEW && !hasIndexId) {
    +            return table;
    +        }
    +        boolean isDiverged = isDivergedView(table);
    +        // here you combine columns from the parent tables the logic is as follows, if the PColumn
    +        // is in the EXCLUDED_COLUMNS remove it, otherwise priority of keeping duplicate columns is
    +        // child -> parent
    +        List<TableInfo> ancestorList = Lists.newArrayList();
    +        TableViewFinderResult viewFinderResult = new TableViewFinderResult();
    +        if (PTableType.VIEW == table.getType()) {
    +            findAncestorViews(tenantId, schemaName, tableName, viewFinderResult,
    +                table.isNamespaceMapped());
    +        } else { // is a view index
    +            findAncestorViewsOfIndex(tenantId, schemaName, tableName, viewFinderResult,
    +                table.isNamespaceMapped());
    +        }
    +        if (viewFinderResult.getResults().isEmpty()) {
    +            // no need to combine columns for local indexes on regular tables
    +            return table;
    +        }
    +        for (TableInfo viewInfo : viewFinderResult.getResults()) {
    +            ancestorList.add(viewInfo);
    +        }
    +        List<PColumn> allColumns = Lists.newArrayList();
    +        List<PColumn> excludedColumns = Lists.newArrayList();
    +        // add my own columns first in reverse order
    +        List<PColumn> myColumns = table.getColumns();
    +        for (int i = myColumns.size() - 1; i >= 0; i--) {
    +            PColumn pColumn = myColumns.get(i);
    +            if (pColumn.isExcluded()) {
    +                excludedColumns.add(pColumn);
    +            } else if (!pColumn.equals(SaltingUtil.SALTING_COLUMN)) { 
    --- End diff --
    
    Instead of matching on SALTING_COLUMN, we should stop the loop at 1 above if table.getSaltBuckets() != null. The code never filters the salt column based on it's name.


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(COLUMN_DEF), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SQL_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATA_TYPE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SQL_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SQL_DATETIME_SUB_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATETIME_SUB),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SQL_DATETIME_SUB), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl CHAR_OCTET_LENGTH_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, Bytes.toBytes(CHAR_OCTET_LENGTH), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl ORDINAL_POSITION_COLUMN = new PColumnImpl(PNameFactory.newName(ORDINAL_POSITION),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, ORDINAL_POSITION_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl IS_NULLABLE_COLUMN = new PColumnImpl(PNameFactory.newName(IS_NULLABLE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(IS_NULLABLE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_CATALOG_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_CATALOG),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_SCHEMA_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_SCHEMA),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_SCHEMA), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_TABLE_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_TABLE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_TABLE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SOURCE_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SOURCE_DATA_TYPE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SOURCE_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl IS_AUTOINCREMENT_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl ARRAY_SIZE_COLUMN = new PColumnImpl(PNameFactory.newName(ARRAY_SIZE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, ARRAY_SIZE_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl COLUMN_FAMILY_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_FAMILY),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, COLUMN_FAMILY_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl TYPE_ID_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, Bytes.toBytes(TYPE_ID), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl VIEW_CONSTANT_COLUMN = new PColumnImpl(PNameFactory.newName(VIEW_CONSTANT),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarbinary.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, VIEW_CONSTANT_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl MULTI_TENANT_COLUMN = new PColumnImpl(PNameFactory.newName(MULTI_TENANT),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PBoolean.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, MULTI_TENANT_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl KEY_SEQ_COLUMN = new PColumnImpl(PNameFactory.newName(KEY_SEQ),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, KEY_SEQ_BYTES, HConstants.LATEST_TIMESTAMP);
    +	
    +	private static final List<PColumnImpl> PK_DATUM_LIST = Lists.newArrayList(TENANT_ID_COLUMN, TABLE_SCHEM_COLUMN, TABLE_NAME_COLUMN, COLUMN_NAME_COLUMN);
    +	
    +	private static final RowProjector GET_COLUMNS_ROW_PROJECTOR = new RowProjector(
    +			Arrays.<ColumnProjector> asList(
    +					new ExpressionProjector(TABLE_CAT, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TENANT_ID_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 0)), false),
    +					new ExpressionProjector(TABLE_SCHEM, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TABLE_SCHEM_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 1)), false),
    +					new ExpressionProjector(TABLE_NAME, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TABLE_NAME_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 2)), false),
    +					new ExpressionProjector(COLUMN_NAME, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(COLUMN_NAME_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 3)), false),
    +					new ExpressionProjector(DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(TYPE_NAME, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(TYPE_NAME_COLUMN), false),
    +					new ExpressionProjector(COLUMN_SIZE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_SIZE_COLUMN), false),
    +					new ExpressionProjector(BUFFER_LENGTH, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(BUFFER_LENGTH_COLUMN), false),
    +					new ExpressionProjector(DECIMAL_DIGITS, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(DECIMAL_DIGITS_COLUMN), false),
    +					new ExpressionProjector(NUM_PREC_RADIX, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(NUM_PREC_RADIX_COLUMN), false),
    +					new ExpressionProjector(NULLABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(NULLABLE_COLUMN), false),
    +					new ExpressionProjector(REMARKS, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(REMARKS_COLUMN), false),
    +					new ExpressionProjector(COLUMN_DEF, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_DEF_COLUMN), false),
    +					new ExpressionProjector(SQL_DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SQL_DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(SQL_DATETIME_SUB, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SQL_DATETIME_SUB_COLUMN), false),
    +					new ExpressionProjector(CHAR_OCTET_LENGTH, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(CHAR_OCTET_LENGTH_COLUMN), false),
    +					new ExpressionProjector(ORDINAL_POSITION, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(ORDINAL_POSITION_COLUMN), false),
    +					new ExpressionProjector(IS_NULLABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(IS_NULLABLE_COLUMN), false),
    +					new ExpressionProjector(SCOPE_CATALOG, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_CATALOG_COLUMN), false),
    +					new ExpressionProjector(SCOPE_SCHEMA, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_SCHEMA_COLUMN), false),
    +					new ExpressionProjector(SCOPE_TABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_TABLE_COLUMN), false),
    +					new ExpressionProjector(SOURCE_DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SOURCE_DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(IS_AUTOINCREMENT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(IS_AUTOINCREMENT_COLUMN), false),
    +					new ExpressionProjector(ARRAY_SIZE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(ARRAY_SIZE_COLUMN), false),
    +					new ExpressionProjector(COLUMN_FAMILY, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_FAMILY_COLUMN), false),
    +					new ExpressionProjector(TYPE_ID, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(TYPE_ID_COLUMN), false),
    +					new ExpressionProjector(VIEW_CONSTANT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(VIEW_CONSTANT_COLUMN), false),
    +					new ExpressionProjector(MULTI_TENANT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(MULTI_TENANT_COLUMN), false),
    +					new ExpressionProjector(KEY_SEQ, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(KEY_SEQ_COLUMN), false)
    +					), 0, true);
    +	
    +	private boolean match(String str, String pattern) throws SQLException {
    +		LiteralExpression strExpr = LiteralExpression.newConstant(str, PVarchar.INSTANCE, SortOrder.ASC);
    +		LiteralExpression patternExpr = LiteralExpression.newConstant(pattern, PVarchar.INSTANCE, SortOrder.ASC);
    +		List<Expression> children = Arrays.<Expression>asList(strExpr, patternExpr);
    +		LikeExpression likeExpr = StringBasedLikeExpression.create(children, LikeType.CASE_SENSITIVE);
    +		ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    +		boolean evaluated = likeExpr.evaluate(null, ptr);
    +		Boolean result = (Boolean)likeExpr.getDataType().toObject(ptr);
    +		if (evaluated) {
    +			return result;
    +		}
    +		return false;
    +	}
    +    
    +	@Override
         public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
                 throws SQLException {
    -        StringBuilder buf = new StringBuilder("select \n " +
    -                TENANT_ID + " " + TABLE_CAT + "," + // use this for tenant id
    -                TABLE_SCHEM + "," +
    -                TABLE_NAME + " ," +
    -                COLUMN_NAME + "," +
    -                ExternalSqlTypeIdFunction.NAME + "(" + DATA_TYPE + ") AS " + DATA_TYPE + "," +
    -                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + "," +
    -                COLUMN_SIZE + "," +
    -                BUFFER_LENGTH + "," +
    -                DECIMAL_DIGITS + "," +
    -                NUM_PREC_RADIX + "," +
    -                NULLABLE + "," +
    -                REMARKS + "," +
    -                COLUMN_DEF + "," +
    -                SQL_DATA_TYPE + "," +
    -                SQL_DATETIME_SUB + "," +
    -                CHAR_OCTET_LENGTH + "," +
    -                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + ORDINAL_POSITION + "-1 ELSE " + ORDINAL_POSITION + " END AS " + ORDINAL_POSITION + "," +
    -                "CASE " + NULLABLE + " WHEN " + DatabaseMetaData.attributeNoNulls +  " THEN '" + Boolean.FALSE.toString() + "' WHEN " + DatabaseMetaData.attributeNullable + " THEN '" + Boolean.TRUE.toString() + "' END AS " + IS_NULLABLE + "," +
    -                SCOPE_CATALOG + "," +
    -                SCOPE_SCHEMA + "," +
    -                SCOPE_TABLE + "," +
    -                SOURCE_DATA_TYPE + "," +
    -                IS_AUTOINCREMENT + "," +
    -                ARRAY_SIZE + "," +
    -                COLUMN_FAMILY + "," +
    -                DATA_TYPE + " " + TYPE_ID + "," +// raw type id for potential internal consumption
    -                VIEW_CONSTANT + "," +
    -                MULTI_TENANT + "," +
    -                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + KEY_SEQ + "-1 ELSE " + KEY_SEQ + " END AS " + KEY_SEQ +
    -                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS + "(" + TENANT_POS_SHIFT + " BOOLEAN)");
    -        StringBuilder where = new StringBuilder();
    -        addTenantIdFilter(where, catalog);
    -        if (schemaPattern != null) {
    -            appendConjunction(where);
    -            where.append(TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'" ));
    -        }
    -        if (tableNamePattern != null && tableNamePattern.length() > 0) {
    -            appendConjunction(where);
    -            where.append(TABLE_NAME + " like '" + StringUtil.escapeStringConstant(tableNamePattern) + "'" );
    -        }
    -        // Allow a "." in columnNamePattern for column family match
    -        String colPattern = null;
    -        if (columnNamePattern != null && columnNamePattern.length() > 0) {
    +		boolean isTenantSpecificConnection = connection.getTenantId() != null;
    +    	List<Tuple> tuples = Lists.newArrayListWithExpectedSize(10);
    +    	ResultSet rs = getTables(catalog, schemaPattern, tableNamePattern, null);
    +    	while (rs.next()) {
    +    		String schemaName = rs.getString(TABLE_SCHEM);
    +    		boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
    +    		// Allow a "." in columnNamePattern for column family match
    +            String colPattern = null;
                 String cfPattern = null;
    -            int index = columnNamePattern.indexOf('.');
    -            if (index <= 0) {
    -                colPattern = columnNamePattern;
    -            } else {
    -                cfPattern = columnNamePattern.substring(0, index);
    -                if (columnNamePattern.length() > index+1) {
    -                    colPattern = columnNamePattern.substring(index+1);
    +            if (columnNamePattern != null && columnNamePattern.length() > 0) {
    +                int index = columnNamePattern.indexOf('.');
    +                if (index <= 0) {
    +                    colPattern = columnNamePattern;
    +                } else {
    +                    cfPattern = columnNamePattern.substring(0, index);
    +                    if (columnNamePattern.length() > index+1) {
    +                        colPattern = columnNamePattern.substring(index+1);
    +                    }
                     }
                 }
    -            if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
    -                // Will pick up only KV columns
    -                appendConjunction(where);
    -                where.append(COLUMN_FAMILY + " like '" + StringUtil.escapeStringConstant(cfPattern) + "'" );
    -            }
    -            if (colPattern != null && colPattern.length() > 0) {
    -                appendConjunction(where);
    -                where.append(COLUMN_NAME + " like '" + StringUtil.escapeStringConstant(colPattern) + "'" );
    -            }
    -        }
    -        if (colPattern == null || colPattern.length() == 0) {
    -            appendConjunction(where);
    -            where.append(COLUMN_NAME + " is not null" );
    -            appendConjunction(where);
    -            where.append(LINK_TYPE + " is null" );
    -        }
    -        boolean isTenantSpecificConnection = connection.getTenantId() != null;
    -        if (isTenantSpecificConnection) {
    -            buf.append(" where (" + where + ") OR ("
    -                    + COLUMN_FAMILY + " is null AND " +  COLUMN_NAME + " is null)");
    -        } else {
    -            buf.append(" where " + where);
    -        }
    -        buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME + "," + SYSTEM_CATALOG_ALIAS + "." + ORDINAL_POSITION);
    -
    -        Statement stmt;
    -        if (isTenantSpecificConnection) {
    -            stmt = connection.createStatement(new PhoenixStatementFactory() {
    -                @Override
    -                public PhoenixStatement newStatement(PhoenixConnection connection) {
    -                    return new PhoenixStatement(connection) {
    -                        @Override
    -                        public PhoenixResultSet newResultSet(ResultIterator iterator, RowProjector projector,
    -                                StatementContext context) throws SQLException {
    -                            return new PhoenixResultSet(new TenantColumnFilteringIterator(iterator, projector),
    -                                    projector, context);
    -                        }
    -                    };
    +			String tableName = rs.getString(TABLE_NAME);
    +    		String tenantId = rs.getString(TABLE_CAT);
    +    		String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
    +    		// if its a SYSTEM table we prevent the rpc to fetch the table so look it up in the cache
    +			PTable table = systemTable ? PhoenixRuntime.getTable(connection, fullTableName)
    +					: PhoenixRuntime.getTableNoCache(connection, fullTableName);
    --- End diff --
    
    Updated to always use cache.


---

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

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<PTable, MetaDataProtos.MutationCode> 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<PTable, MetaDataProtos.MutationCode>(table,
    +                    MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
    +        }
    +        if (!skipAddingParentColumns) {
    +            table =
    +                    addDerivedColumnsFromAncestors(table, timestamp, clientVersion,
    +                        lockedAncestorTable);
    +            if (table==null) {
    +                return new Pair<PTable, MetaDataProtos.MutationCode>(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<PTable> 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<PTable, MetaDataProtos.MutationCode>(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()
    +                        : ByteUtil.EMPTY_BYTE_ARRAY;
    +        byte[] schemaName = table.getSchemaName().getBytes();
    +        byte[] tableName = table.getTableName().getBytes();
    +		String fullTableName = SchemaUtil.getTableName(table.getSchemaName().getString(),
    +				table.getTableName().getString());
    +        boolean hasIndexId = table.getViewIndexId() != null;
    +        boolean isSalted = table.getBucketNum() != null;
    +        if (table.getType() != PTableType.VIEW && !hasIndexId) {
    +            return table;
    +        }
    +        boolean isDiverged = isDivergedView(table);
    +        // here you combine columns from the parent tables the logic is as follows, if the PColumn
    +        // is in the EXCLUDED_COLUMNS remove it, otherwise priority of keeping duplicate columns is
    +        // child -> parent
    +        List<TableInfo> ancestorList = Lists.newArrayList();
    +        TableViewFinderResult viewFinderResult = new TableViewFinderResult();
    +        if (PTableType.VIEW == table.getType()) {
    +            findAncestorViews(tenantId, schemaName, tableName, viewFinderResult,
    +                table.isNamespaceMapped());
    +        } else { // is a view index
    +            findAncestorViewsOfIndex(tenantId, schemaName, tableName, viewFinderResult,
    +                table.isNamespaceMapped());
    +        }
    +        if (viewFinderResult.getResults().isEmpty()) {
    +            // no need to combine columns for local indexes on regular tables
    +            return table;
    +        }
    +        for (TableInfo viewInfo : viewFinderResult.getResults()) {
    +            ancestorList.add(viewInfo);
    +        }
    +        List<PColumn> allColumns = Lists.newArrayList();
    +        List<PColumn> excludedColumns = Lists.newArrayList();
    +        // add my own columns first in reverse order
    +        List<PColumn> myColumns = table.getColumns();
    +        for (int i = myColumns.size() - 1; i >= 0; i--) {
    +            PColumn pColumn = myColumns.get(i);
    +            if (pColumn.isExcluded()) {
    +                excludedColumns.add(pColumn);
    +            } else if (!pColumn.equals(SaltingUtil.SALTING_COLUMN)) { 
    --- End diff --
    
    I will change this.


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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<Mutation> 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 ...

Posted by gjacoby126 <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<PTable, MetaDataProtos.MutationCode> 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<PTable, MetaDataProtos.MutationCode>(table,
    +                    MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
    +        }
    +        if (!skipAddingParentColumns) {
    +            table =
    +                    addDerivedColumnsFromAncestors(table, timestamp, clientVersion,
    +                        lockedAncestorTable);
    +            if (table==null) {
    +                return new Pair<PTable, MetaDataProtos.MutationCode>(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<PTable> 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<PTable, MetaDataProtos.MutationCode>(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()
    +                        : ByteUtil.EMPTY_BYTE_ARRAY;
    +        byte[] schemaName = table.getSchemaName().getBytes();
    +        byte[] tableName = table.getTableName().getBytes();
    +		String fullTableName = SchemaUtil.getTableName(table.getSchemaName().getString(),
    +				table.getTableName().getString());
    +        boolean hasIndexId = table.getViewIndexId() != null;
    +        boolean isSalted = table.getBucketNum() != null;
    +        if (table.getType() != PTableType.VIEW && !hasIndexId) {
    +            return table;
    +        }
    +        boolean isDiverged = isDivergedView(table);
    +        // here you combine columns from the parent tables the logic is as follows, if the PColumn
    +        // is in the EXCLUDED_COLUMNS remove it, otherwise priority of keeping duplicate columns is
    +        // child -> parent
    +        List<TableInfo> ancestorList = Lists.newArrayList();
    +        TableViewFinderResult viewFinderResult = new TableViewFinderResult();
    +        if (PTableType.VIEW == table.getType()) {
    +            findAncestorViews(tenantId, schemaName, tableName, viewFinderResult,
    +                table.isNamespaceMapped());
    +        } else { // is a view index
    +            findAncestorViewsOfIndex(tenantId, schemaName, tableName, viewFinderResult,
    +                table.isNamespaceMapped());
    +        }
    +        if (viewFinderResult.getResults().isEmpty()) {
    +            // no need to combine columns for local indexes on regular tables
    +            return table;
    +        }
    +        for (TableInfo viewInfo : viewFinderResult.getResults()) {
    +            ancestorList.add(viewInfo);
    +        }
    +        List<PColumn> allColumns = Lists.newArrayList();
    +        List<PColumn> excludedColumns = Lists.newArrayList();
    +        // add my own columns first in reverse order
    +        List<PColumn> myColumns = table.getColumns();
    +        for (int i = myColumns.size() - 1; i >= 0; i--) {
    +            PColumn pColumn = myColumns.get(i);
    +            if (pColumn.isExcluded()) {
    +                excludedColumns.add(pColumn);
    +            } else if (!pColumn.equals(SaltingUtil.SALTING_COLUMN)) { 
    +                // skip salted column as it will be added from the base table columns
    +                allColumns.add(pColumn);
    +            }
    +        }
    +
    +        // initialize map from with indexed expression to list of required data columns
    +        // then remove the data columns that have not been dropped, so that we get the columns that
    +        // have been dropped
    +        Map<PColumn, List<String>> indexRequiredDroppedDataColMap =
    +                Maps.newHashMapWithExpectedSize(table.getColumns().size());
    +        if (hasIndexId) {
    +            int indexPosOffset = (isSalted ? 1 : 0) + (table.isMultiTenant() ? 1 : 0) + 1;
    +            ColumnNameTrackingExpressionCompiler expressionCompiler =
    +                    new ColumnNameTrackingExpressionCompiler();
    +            for (int i = indexPosOffset; i < table.getPKColumns().size(); i++) {
    +                PColumn indexColumn = table.getPKColumns().get(i);
    +                try {
    +                    expressionCompiler.reset();
    +                    String expressionStr = IndexUtil.getIndexColumnExpressionStr(indexColumn);
    +                    ParseNode parseNode = SQLParser.parseCondition(expressionStr);
    +                    parseNode.accept(expressionCompiler);
    +                    indexRequiredDroppedDataColMap.put(indexColumn,
    +                        Lists.newArrayList(expressionCompiler.getDataColumnNames()));
    +                } catch (SQLException e) {
    +                    throw new RuntimeException(e); // Impossible
    +                }
    +            }
    +        }
    +
    +        // now go up from child to parent all the way to the base table:
    +        PTable baseTable = null;
    +        long maxTableTimestamp = -1;
    +        int numPKCols = table.getPKColumns().size();
    +        for (int i = 0; i < ancestorList.size(); i++) {
    +            TableInfo parentTableInfo = ancestorList.get(i);
    +            PTable pTable = null;
    +            String fullParentTableName = SchemaUtil.getTableName(parentTableInfo.getSchemaName(),
    +			    parentTableInfo.getTableName());
    +            PName parentTenantId =
    +                    (parentTableInfo.getTenantId() != null && parentTableInfo.getTenantId().length!=0)
    +                            ? PNameFactory.newName(parentTableInfo.getTenantId()) : null;
    +            PTableKey pTableKey = new PTableKey(parentTenantId, fullParentTableName);
    +            // if we already have the PTable of an ancestor that has been locked, no need to look up
    +            // the table
    +            if (lockedAncestorTable != null && lockedAncestorTable.getKey().equals(pTableKey)) {
    +                pTable = lockedAncestorTable;
    +            } else {
    +                // if we are currently combining columns for a view index and are looking up its
    +                // ancestors we do not add the indexes to the ancestor PTable (or else we end up in
    +                // a circular loop)
    +                // we also don't need to add parent columns of the ancestors as we combine columns
    +                // from all ancestors
    +                pTable =
    +                        doGetTable(parentTableInfo.getTenantId(), parentTableInfo.getSchemaName(),
    +                            parentTableInfo.getTableName(), timestamp, null, clientVersion, hasIndexId,
    +                            true, null);
    +            }
    +            if (pTable == null) {
    +                throw new ParentTableNotFoundException(parentTableInfo, fullTableName);
    +            } else {
    +                // only combine columns for view indexes (and not local indexes on regular tables
    +                // which also have a viewIndexId)
    +                if (i == 0 && hasIndexId && pTable.getType() != PTableType.VIEW) {
    +                    return table;
    +                }
    +                if (TABLE.equals(pTable.getType())) {
    +                    baseTable = pTable;
    +                }
    +                // set the final table timestamp as the max timestamp of the view/view index or its
    +                // ancestors
    +                maxTableTimestamp = Math.max(maxTableTimestamp, pTable.getTimeStamp());
    +                if (hasIndexId) {
    +                    // add all pk columns of parent tables to indexes
    +                    for (PColumn column : pTable.getPKColumns()) {
    +                        // don't add the salt column of ancestor tables for view indexes
    +                        if (column.equals(SaltingUtil.SALTING_COLUMN) || column.isExcluded()) {
    --- End diff --
    
    I will change this.


---

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

Posted by twdsilva <gi...@git.apache.org>.
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.
    -        if (!isMultiTenant) {
    -            byte[] startRow = ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY);
    -            byte[] stopRow = ByteUtil.nextKey(startRow);
    -            scan.setStartRow(startRow);
    -            scan.setStopRow(stopRow);
    -        }
    -        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, linkTypeBytes);
    -        SingleColumnValueFilter tableTypeFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES,
    -                CompareOp.EQUAL, PTableType.VIEW.getSerializedValue().getBytes());
    -        tableTypeFilter.setFilterIfMissing(false);
    -        linkFilter.setFilterIfMissing(true);
    -        byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil
    -                .getPhysicalHBaseTableName(schemaName, tableName, table.isNamespaceMapped())
    -                .getBytes());
    -        SuffixFilter rowFilter = new SuffixFilter(suffix);
    -        List<Filter> filters = Lists.<Filter>newArrayList(linkFilter,tableTypeFilter,rowFilter);
    -        if (stopAfterFirst) {
    -            filters.add(new PageFilter(1));
    -        }
    -        FilterList filter = new FilterList(filters);
    -        scan.setFilter(filter);
    -        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
    -        
    -        // Original region-only scanner modified due to PHOENIX-1208
    -        // RegionScanner scanner = region.getScanner(scan);
    -        // The following *should* work, but doesn't due to HBASE-11837
    -        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
    -        // HTableInterface hTable = env.getTable(systemCatalogTableName);
    -        // These deprecated calls work around the issue
    -        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
    -            region.getTableDesc().getTableName().getName())) {
    -            boolean allViewsInCurrentRegion = true;
    -            int numOfChildViews = 0;
    -            List<ViewInfo> viewInfoList = Lists.newArrayList();
    -            try (ResultScanner scanner = hTable.getScanner(scan)) {
    -                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
    -                    numOfChildViews++;
    -                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    -                    ResultTuple resultTuple = new ResultTuple(result);
    -                    resultTuple.getKey(ptr);
    -                    byte[] key = ptr.copyBytes();
    -                    if (checkTableKeyInRegion(key, region) != null) {
    -                        allViewsInCurrentRegion = false;
    -                    }
    -                    byte[][] rowKeyMetaData = new byte[3][];
    -                    getVarChars(result.getRow(), 3, rowKeyMetaData);
    -                    byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    -                    byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    -                    byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    -                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
    -                }
    -                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
    -                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
    -                    tableViewFinderResult.setAllViewsNotInSingleRegion();
    -                }
    -                return tableViewFinderResult;
    +
    +    private void findAncestorViewsOfIndex(byte[] tenantId, byte[] schemaName, byte[] indexName,
    --- End diff --
    
    Done.


---

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

Posted by JamesRTaylor <gi...@git.apache.org>.
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<Mutation> tableMetadata = ProtobufUtil.getMutations(request);
    +            List<Mutation> 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<byte[]> tableNamesToDelete = Lists.newArrayList();
                 List<SharedTableState> 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 issue #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG table

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on the issue:

    https://github.com/apache/phoenix/pull/303
  
    @vincentpoon  @gjacoby126  This is the PR to support splittable system.catalog. Please review if you have some time. 


---

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

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/303#discussion_r201748315
  
    --- 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 --
    
    I created PHOENIX-4766 for this, I will add a comment referencing this JIRA in createTableInternal().


---

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

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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.
    -        if (!isMultiTenant) {
    -            byte[] startRow = ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY);
    -            byte[] stopRow = ByteUtil.nextKey(startRow);
    -            scan.setStartRow(startRow);
    -            scan.setStopRow(stopRow);
    -        }
    -        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, linkTypeBytes);
    -        SingleColumnValueFilter tableTypeFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES,
    -                CompareOp.EQUAL, PTableType.VIEW.getSerializedValue().getBytes());
    -        tableTypeFilter.setFilterIfMissing(false);
    -        linkFilter.setFilterIfMissing(true);
    -        byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil
    -                .getPhysicalHBaseTableName(schemaName, tableName, table.isNamespaceMapped())
    -                .getBytes());
    -        SuffixFilter rowFilter = new SuffixFilter(suffix);
    -        List<Filter> filters = Lists.<Filter>newArrayList(linkFilter,tableTypeFilter,rowFilter);
    -        if (stopAfterFirst) {
    -            filters.add(new PageFilter(1));
    -        }
    -        FilterList filter = new FilterList(filters);
    -        scan.setFilter(filter);
    -        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
    -        
    -        // Original region-only scanner modified due to PHOENIX-1208
    -        // RegionScanner scanner = region.getScanner(scan);
    -        // The following *should* work, but doesn't due to HBASE-11837
    -        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
    -        // HTableInterface hTable = env.getTable(systemCatalogTableName);
    -        // These deprecated calls work around the issue
    -        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
    -            region.getTableDesc().getTableName().getName())) {
    -            boolean allViewsInCurrentRegion = true;
    -            int numOfChildViews = 0;
    -            List<ViewInfo> viewInfoList = Lists.newArrayList();
    -            try (ResultScanner scanner = hTable.getScanner(scan)) {
    -                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
    -                    numOfChildViews++;
    -                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    -                    ResultTuple resultTuple = new ResultTuple(result);
    -                    resultTuple.getKey(ptr);
    -                    byte[] key = ptr.copyBytes();
    -                    if (checkTableKeyInRegion(key, region) != null) {
    -                        allViewsInCurrentRegion = false;
    -                    }
    -                    byte[][] rowKeyMetaData = new byte[3][];
    -                    getVarChars(result.getRow(), 3, rowKeyMetaData);
    -                    byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
    -                    byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
    -                    byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
    -                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
    -                }
    -                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
    -                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
    -                    tableViewFinderResult.setAllViewsNotInSingleRegion();
    -                }
    -                return tableViewFinderResult;
    +
    +    private void findAncestorViewsOfIndex(byte[] tenantId, byte[] schemaName, byte[] indexName,
    +            TableViewFinderResult result, boolean isNamespaceMapped) throws IOException {
    +        try (Table hTable =
    +                env.getTable(SchemaUtil.getPhysicalTableName(
    +                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
    +            TableViewFinderResult currentResult =
    +                    ViewFinder.findParentViewofIndex(hTable, tenantId, schemaName, indexName);
    +            if (currentResult.getResults().size() == 1) {
    +                result.addResult(currentResult);
    +                TableInfo tableInfo = currentResult.getResults().get(0);
    +                findAncestorViews(tableInfo.getTenantId(), tableInfo.getSchemaName(),
    +                    tableInfo.getTableName(), result, isNamespaceMapped);
                 }
    +            // else this is an index on a regular table and so we don't need to combine columns
             }
         }
         
    -    private TableViewFinder findChildViews_4_11(Region region, byte[] tenantId, byte[] schemaName, byte[] tableName, boolean stopAfterFirst) throws IOException {
    -        Scan scan = new Scan();
    -        byte[] startRow = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
    -        byte[] stopRow = ByteUtil.nextKey(startRow);
    -        scan.setStartRow(startRow);
    -        scan.setStopRow(stopRow);
    -        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, CHILD_TABLE_BYTES);
    -        Filter filter = linkFilter;
    -        linkFilter.setFilterIfMissing(true);
    -        if (stopAfterFirst) {
    -            filter = new FilterList(linkFilter, new PageFilter(1));
    -        }
    -        scan.setFilter(filter);
    -        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
    -        scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
    -        
    -        // Original region-only scanner modified due to PHOENIX-1208
    -        // RegionScanner scanner = region.getScanner(scan);
    -        // The following *should* work, but doesn't due to HBASE-11837
    -        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
    -        // HTableInterface hTable = env.getTable(systemCatalogTableName);
    -        // These deprecated calls work around the issue
    -        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
    -            region.getTableDesc().getTableName().getName())) {
    -            boolean allViewsInCurrentRegion = true;
    -            int numOfChildViews = 0;
    -            List<ViewInfo> viewInfoList = Lists.newArrayList();
    -            try (ResultScanner scanner = hTable.getScanner(scan)) {
    -                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
    -                    numOfChildViews++;
    -                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    -                    ResultTuple resultTuple = new ResultTuple(result);
    -                    resultTuple.getKey(ptr);
    -                    byte[] key = ptr.copyBytes();
    -                    if (checkTableKeyInRegion(key, region) != null) {
    -                        allViewsInCurrentRegion = false;
    -                    }
    -                    byte[][] rowViewKeyMetaData = new byte[5][];
    -                    getVarChars(result.getRow(), 5, rowViewKeyMetaData);
    -                    byte[] viewTenantId = rowViewKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
    -                    byte[] viewSchemaName = SchemaUtil.getSchemaNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
    -                    byte[] viewName = SchemaUtil.getTableNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
    -                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
    -                }
    -                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
    -                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
    -                    tableViewFinderResult.setAllViewsNotInSingleRegion();
    -                }
    -                return tableViewFinderResult;
    +    private void findAncestorViews(byte[] tenantId, byte[] schemaName, byte[] tableName,
    +            TableViewFinderResult result, boolean isNamespaceMapped) throws IOException {
    +        try (Table hTable =
    +                env.getTable(SchemaUtil.getPhysicalTableName(
    +                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
    +            ViewFinder.findAllRelatives(hTable, tenantId, schemaName, tableName,
    +                LinkType.PARENT_TABLE, result);
    +            if (!isNamespaceMapped || schemaName.length==0) {
    +                // the child->parent link is overwritten by the child->physical table link for first
    +                // level children of base table when namespace mapping is disabled or if the parent
    +                // table doesn't have a schema as both the parent table name and physical table name
    +                // are the same.
    +                // When namespace mapping is enabled the physical table name is of the form S:T
    +                // while the table name is of the form S.T so we need to query for the
    +                // PHYSICAL_TABLE link
    --- End diff --
    
    We are only rewriting the PARENT->CHILD links (removing them from SYSTEM.CATALOG and recreating them in the new SYSTEM.CHILD_LINK table). The PHYSICAL_TABLE link overwriting the PARENT_TABLE link happens in the CHILD->PARENT links. 


---

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

Posted by twdsilva <gi...@git.apache.org>.
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), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(COLUMN_DEF), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SQL_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATA_TYPE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SQL_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SQL_DATETIME_SUB_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATETIME_SUB),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SQL_DATETIME_SUB), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl CHAR_OCTET_LENGTH_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, Bytes.toBytes(CHAR_OCTET_LENGTH), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl ORDINAL_POSITION_COLUMN = new PColumnImpl(PNameFactory.newName(ORDINAL_POSITION),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, ORDINAL_POSITION_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl IS_NULLABLE_COLUMN = new PColumnImpl(PNameFactory.newName(IS_NULLABLE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(IS_NULLABLE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_CATALOG_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_CATALOG),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_SCHEMA_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_SCHEMA),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_SCHEMA), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_TABLE_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_TABLE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_TABLE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SOURCE_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SOURCE_DATA_TYPE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SOURCE_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl IS_AUTOINCREMENT_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl ARRAY_SIZE_COLUMN = new PColumnImpl(PNameFactory.newName(ARRAY_SIZE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, ARRAY_SIZE_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl COLUMN_FAMILY_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_FAMILY),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, COLUMN_FAMILY_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl TYPE_ID_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, Bytes.toBytes(TYPE_ID), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl VIEW_CONSTANT_COLUMN = new PColumnImpl(PNameFactory.newName(VIEW_CONSTANT),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarbinary.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, VIEW_CONSTANT_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl MULTI_TENANT_COLUMN = new PColumnImpl(PNameFactory.newName(MULTI_TENANT),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PBoolean.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, MULTI_TENANT_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl KEY_SEQ_COLUMN = new PColumnImpl(PNameFactory.newName(KEY_SEQ),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, KEY_SEQ_BYTES, HConstants.LATEST_TIMESTAMP);
    +	
    +	private static final List<PColumnImpl> PK_DATUM_LIST = Lists.newArrayList(TENANT_ID_COLUMN, TABLE_SCHEM_COLUMN, TABLE_NAME_COLUMN, COLUMN_NAME_COLUMN);
    +	
    +	private static final RowProjector GET_COLUMNS_ROW_PROJECTOR = new RowProjector(
    +			Arrays.<ColumnProjector> asList(
    +					new ExpressionProjector(TABLE_CAT, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TENANT_ID_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 0)), false),
    +					new ExpressionProjector(TABLE_SCHEM, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TABLE_SCHEM_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 1)), false),
    +					new ExpressionProjector(TABLE_NAME, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TABLE_NAME_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 2)), false),
    +					new ExpressionProjector(COLUMN_NAME, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(COLUMN_NAME_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 3)), false),
    +					new ExpressionProjector(DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(TYPE_NAME, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(TYPE_NAME_COLUMN), false),
    +					new ExpressionProjector(COLUMN_SIZE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_SIZE_COLUMN), false),
    +					new ExpressionProjector(BUFFER_LENGTH, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(BUFFER_LENGTH_COLUMN), false),
    +					new ExpressionProjector(DECIMAL_DIGITS, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(DECIMAL_DIGITS_COLUMN), false),
    +					new ExpressionProjector(NUM_PREC_RADIX, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(NUM_PREC_RADIX_COLUMN), false),
    +					new ExpressionProjector(NULLABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(NULLABLE_COLUMN), false),
    +					new ExpressionProjector(REMARKS, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(REMARKS_COLUMN), false),
    +					new ExpressionProjector(COLUMN_DEF, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_DEF_COLUMN), false),
    +					new ExpressionProjector(SQL_DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SQL_DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(SQL_DATETIME_SUB, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SQL_DATETIME_SUB_COLUMN), false),
    +					new ExpressionProjector(CHAR_OCTET_LENGTH, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(CHAR_OCTET_LENGTH_COLUMN), false),
    +					new ExpressionProjector(ORDINAL_POSITION, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(ORDINAL_POSITION_COLUMN), false),
    +					new ExpressionProjector(IS_NULLABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(IS_NULLABLE_COLUMN), false),
    +					new ExpressionProjector(SCOPE_CATALOG, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_CATALOG_COLUMN), false),
    +					new ExpressionProjector(SCOPE_SCHEMA, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_SCHEMA_COLUMN), false),
    +					new ExpressionProjector(SCOPE_TABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_TABLE_COLUMN), false),
    +					new ExpressionProjector(SOURCE_DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SOURCE_DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(IS_AUTOINCREMENT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(IS_AUTOINCREMENT_COLUMN), false),
    +					new ExpressionProjector(ARRAY_SIZE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(ARRAY_SIZE_COLUMN), false),
    +					new ExpressionProjector(COLUMN_FAMILY, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_FAMILY_COLUMN), false),
    +					new ExpressionProjector(TYPE_ID, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(TYPE_ID_COLUMN), false),
    +					new ExpressionProjector(VIEW_CONSTANT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(VIEW_CONSTANT_COLUMN), false),
    +					new ExpressionProjector(MULTI_TENANT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(MULTI_TENANT_COLUMN), false),
    +					new ExpressionProjector(KEY_SEQ, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(KEY_SEQ_COLUMN), false)
    +					), 0, true);
    +	
    +	private boolean match(String str, String pattern) throws SQLException {
    +		LiteralExpression strExpr = LiteralExpression.newConstant(str, PVarchar.INSTANCE, SortOrder.ASC);
    +		LiteralExpression patternExpr = LiteralExpression.newConstant(pattern, PVarchar.INSTANCE, SortOrder.ASC);
    +		List<Expression> children = Arrays.<Expression>asList(strExpr, patternExpr);
    +		LikeExpression likeExpr = StringBasedLikeExpression.create(children, LikeType.CASE_SENSITIVE);
    +		ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    +		boolean evaluated = likeExpr.evaluate(null, ptr);
    +		Boolean result = (Boolean)likeExpr.getDataType().toObject(ptr);
    +		if (evaluated) {
    +			return result;
    +		}
    +		return false;
    +	}
    +    
    +	@Override
         public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
                 throws SQLException {
    -        StringBuilder buf = new StringBuilder("select \n " +
    -                TENANT_ID + " " + TABLE_CAT + "," + // use this for tenant id
    -                TABLE_SCHEM + "," +
    -                TABLE_NAME + " ," +
    -                COLUMN_NAME + "," +
    -                ExternalSqlTypeIdFunction.NAME + "(" + DATA_TYPE + ") AS " + DATA_TYPE + "," +
    -                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + "," +
    -                COLUMN_SIZE + "," +
    -                BUFFER_LENGTH + "," +
    -                DECIMAL_DIGITS + "," +
    -                NUM_PREC_RADIX + "," +
    -                NULLABLE + "," +
    -                REMARKS + "," +
    -                COLUMN_DEF + "," +
    -                SQL_DATA_TYPE + "," +
    -                SQL_DATETIME_SUB + "," +
    -                CHAR_OCTET_LENGTH + "," +
    -                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + ORDINAL_POSITION + "-1 ELSE " + ORDINAL_POSITION + " END AS " + ORDINAL_POSITION + "," +
    -                "CASE " + NULLABLE + " WHEN " + DatabaseMetaData.attributeNoNulls +  " THEN '" + Boolean.FALSE.toString() + "' WHEN " + DatabaseMetaData.attributeNullable + " THEN '" + Boolean.TRUE.toString() + "' END AS " + IS_NULLABLE + "," +
    -                SCOPE_CATALOG + "," +
    -                SCOPE_SCHEMA + "," +
    -                SCOPE_TABLE + "," +
    -                SOURCE_DATA_TYPE + "," +
    -                IS_AUTOINCREMENT + "," +
    -                ARRAY_SIZE + "," +
    -                COLUMN_FAMILY + "," +
    -                DATA_TYPE + " " + TYPE_ID + "," +// raw type id for potential internal consumption
    -                VIEW_CONSTANT + "," +
    -                MULTI_TENANT + "," +
    -                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + KEY_SEQ + "-1 ELSE " + KEY_SEQ + " END AS " + KEY_SEQ +
    -                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS + "(" + TENANT_POS_SHIFT + " BOOLEAN)");
    -        StringBuilder where = new StringBuilder();
    -        addTenantIdFilter(where, catalog);
    -        if (schemaPattern != null) {
    -            appendConjunction(where);
    -            where.append(TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'" ));
    -        }
    -        if (tableNamePattern != null && tableNamePattern.length() > 0) {
    -            appendConjunction(where);
    -            where.append(TABLE_NAME + " like '" + StringUtil.escapeStringConstant(tableNamePattern) + "'" );
    -        }
    -        // Allow a "." in columnNamePattern for column family match
    -        String colPattern = null;
    -        if (columnNamePattern != null && columnNamePattern.length() > 0) {
    +		boolean isTenantSpecificConnection = connection.getTenantId() != null;
    +    	List<Tuple> tuples = Lists.newArrayListWithExpectedSize(10);
    +    	ResultSet rs = getTables(catalog, schemaPattern, tableNamePattern, null);
    +    	while (rs.next()) {
    +    		String schemaName = rs.getString(TABLE_SCHEM);
    +    		boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
    +    		// Allow a "." in columnNamePattern for column family match
    +            String colPattern = null;
                 String cfPattern = null;
    -            int index = columnNamePattern.indexOf('.');
    -            if (index <= 0) {
    -                colPattern = columnNamePattern;
    -            } else {
    -                cfPattern = columnNamePattern.substring(0, index);
    -                if (columnNamePattern.length() > index+1) {
    -                    colPattern = columnNamePattern.substring(index+1);
    +            if (columnNamePattern != null && columnNamePattern.length() > 0) {
    +                int index = columnNamePattern.indexOf('.');
    +                if (index <= 0) {
    +                    colPattern = columnNamePattern;
    +                } else {
    +                    cfPattern = columnNamePattern.substring(0, index);
    +                    if (columnNamePattern.length() > index+1) {
    +                        colPattern = columnNamePattern.substring(index+1);
    +                    }
                     }
                 }
    -            if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
    -                // Will pick up only KV columns
    -                appendConjunction(where);
    -                where.append(COLUMN_FAMILY + " like '" + StringUtil.escapeStringConstant(cfPattern) + "'" );
    -            }
    -            if (colPattern != null && colPattern.length() > 0) {
    -                appendConjunction(where);
    -                where.append(COLUMN_NAME + " like '" + StringUtil.escapeStringConstant(colPattern) + "'" );
    -            }
    -        }
    -        if (colPattern == null || colPattern.length() == 0) {
    -            appendConjunction(where);
    -            where.append(COLUMN_NAME + " is not null" );
    -            appendConjunction(where);
    -            where.append(LINK_TYPE + " is null" );
    -        }
    -        boolean isTenantSpecificConnection = connection.getTenantId() != null;
    -        if (isTenantSpecificConnection) {
    -            buf.append(" where (" + where + ") OR ("
    -                    + COLUMN_FAMILY + " is null AND " +  COLUMN_NAME + " is null)");
    -        } else {
    -            buf.append(" where " + where);
    -        }
    -        buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME + "," + SYSTEM_CATALOG_ALIAS + "." + ORDINAL_POSITION);
    -
    -        Statement stmt;
    -        if (isTenantSpecificConnection) {
    -            stmt = connection.createStatement(new PhoenixStatementFactory() {
    -                @Override
    -                public PhoenixStatement newStatement(PhoenixConnection connection) {
    -                    return new PhoenixStatement(connection) {
    -                        @Override
    -                        public PhoenixResultSet newResultSet(ResultIterator iterator, RowProjector projector,
    -                                StatementContext context) throws SQLException {
    -                            return new PhoenixResultSet(new TenantColumnFilteringIterator(iterator, projector),
    -                                    projector, context);
    -                        }
    -                    };
    +			String tableName = rs.getString(TABLE_NAME);
    +    		String tenantId = rs.getString(TABLE_CAT);
    +    		String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
    +    		// if its a SYSTEM table we prevent the rpc to fetch the table so look it up in the cache
    +			PTable table = systemTable ? PhoenixRuntime.getTable(connection, fullTableName)
    +					: PhoenixRuntime.getTableNoCache(connection, fullTableName);
    +    		boolean isSalted = table.getBucketNum()!=null;
    +    		boolean tenantColSkipped = false;
    +    		for (PColumn column : table.getColumns()) {
    +    			if (isTenantSpecificConnection && column.equals(table.getPKColumns().get(0))) {
    +    				// skip the tenant column
    +    				tenantColSkipped = true;
    +    				continue;
    +    			}
    +    			String columnFamily = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
    +    			String columnName = column.getName().getString();
    +    			if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
    +    				if (columnFamily==null || !match(columnFamily, cfPattern)) {
    +    					continue;
    +    				}
                     }
    -            });
    -        } else {
    -            stmt = connection.createStatement();
    -        }
    -        return stmt.executeQuery(buf.toString());
    -    }
    -    
    -//    private ColumnResolver getColumnResolverForCatalogTable() throws SQLException {
    -//        TableRef tableRef = new TableRef(getTable(connection, SYSTEM_CATALOG_NAME));
    -//        return FromCompiler.getResolver(tableRef);
    -//    }
    -    
    -    /**
    -     * Filters the tenant id column out of a column metadata result set (thus, where each row is a column definition).
    -     * The tenant id is by definition the first column of the primary key, but the primary key does not necessarily
    -     * start at the first column. Assumes columns are sorted on ordinal position.
    -     */
    -    private static class TenantColumnFilteringIterator extends DelegateResultIterator {
    -        private final RowProjector rowProjector;
    -        private final int columnFamilyIndex;
    -        private final int columnNameIndex;
    -        private final int multiTenantIndex;
    -        private final int keySeqIndex;
    -        private boolean inMultiTenantTable;
    -        private boolean tenantColumnSkipped;
    -
    -        private TenantColumnFilteringIterator(ResultIterator delegate, RowProjector rowProjector) throws SQLException {
    -            super(delegate);
    -            this.rowProjector = rowProjector;
    -            this.columnFamilyIndex = rowProjector.getColumnIndex(COLUMN_FAMILY);
    -            this.columnNameIndex = rowProjector.getColumnIndex(COLUMN_NAME);
    -            this.multiTenantIndex = rowProjector.getColumnIndex(MULTI_TENANT);
    -            this.keySeqIndex = rowProjector.getColumnIndex(KEY_SEQ);
    -        }
    -
    -        @Override
    -        public Tuple next() throws SQLException {
    -            Tuple tuple = super.next();
    -
    -            while (tuple != null
    -                    && getColumn(tuple, columnFamilyIndex) == null && getColumn(tuple, columnNameIndex) == null) {
    -                // new table, check if it is multitenant
    -                inMultiTenantTable = getColumn(tuple, multiTenantIndex) == Boolean.TRUE;
    -                tenantColumnSkipped = false;
    -                // skip row representing table
    -                tuple = super.next();
    -            }
    -
    -            if (tuple != null && inMultiTenantTable && !tenantColumnSkipped) {
    -                Object value = getColumn(tuple, keySeqIndex);
    -                if (value != null && ((Number)value).longValue() == 1L) {
    -                    tenantColumnSkipped = true;
    -                    // skip tenant id primary key column
    -                    return next();
    +                if (colPattern != null && colPattern.length() > 0) {
    +                	if (!match(columnName, colPattern)) {
    +                		continue;
    +                	}
                     }
    -            }
    -
    -            if (tuple != null && tenantColumnSkipped) {
    -                ResultTuple resultTuple = (ResultTuple)tuple;
    -                List<Cell> cells = resultTuple.getResult().listCells();
    -                KeyValue kv = new KeyValue(resultTuple.getResult().getRow(), TABLE_FAMILY_BYTES,
    -                        TENANT_POS_SHIFT_BYTES, PDataType.TRUE_BYTES);
    -                List<Cell> newCells = Lists.newArrayListWithCapacity(cells.size() + 1);
    -                newCells.addAll(cells);
    -                newCells.add(kv);
    -                Collections.sort(newCells, KeyValue.COMPARATOR);
    -                tuple = new ResultTuple(Result.create(newCells));
    -            }
    -            return tuple;
    -        }
    -
    -        private Object getColumn(Tuple tuple, int index) throws SQLException {
    -            ColumnProjector projector = this.rowProjector.getColumnProjector(index);
    -            PDataType type = projector.getExpression().getDataType();
    -            return projector.getValue(tuple, type, new ImmutableBytesPtr());
    -        }
    +                // generate row key
    +                // TENANT_ID, TABLE_SCHEM, TABLE_NAME , COLUMN_NAME are row key columns
    +                byte[] rowKey =
    +                        SchemaUtil.getColumnKey(tenantId, schemaName, tableName, columnName, null);
    +
    +                // add one cell for each column info
    +                List<Cell> cells = Lists.newArrayListWithCapacity(25);
    +                // DATA_TYPE
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, DATA_TYPE_BYTES,
    +                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    PInteger.INSTANCE.toBytes(column.getDataType().getResultSetSqlType())));
    +                // TYPE_NAME
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
    +                    Bytes.toBytes(TYPE_NAME), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    column.getDataType().getSqlTypeNameBytes()));
    +                // COLUMN_SIZE
    +                cells.add(
    +                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, COLUMN_SIZE_BYTES,
    +                        MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                        column.getMaxLength() != null
    +                                ? PInteger.INSTANCE.toBytes(column.getMaxLength())
    +                                : ByteUtil.EMPTY_BYTE_ARRAY));
    +                // BUFFER_LENGTH
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
    +                    Bytes.toBytes(BUFFER_LENGTH), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    ByteUtil.EMPTY_BYTE_ARRAY));
    +                // DECIMAL_DIGITS
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES,
    +                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    column.getScale() != null ? PInteger.INSTANCE.toBytes(column.getScale())
    +                            : ByteUtil.EMPTY_BYTE_ARRAY));
    +                // NUM_PREC_RADIX
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
    +                    Bytes.toBytes(NUM_PREC_RADIX), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    ByteUtil.EMPTY_BYTE_ARRAY));
    +                // NULLABLE
    +                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, NULLABLE_BYTES,
    +                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
    +                    PInteger.INSTANCE.toBytes(SchemaUtil.getIsNullableInt(column.isNullable()))));
    +                // REMARKS
    +                cells.add(
    +                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, Bytes.toBytes(REMARKS),
    +                        MetaDataProtocol.MIN_TABLE_TIMESTAMP, ByteUtil.EMPTY_BYTE_ARRAY));
    --- End diff --
    
    QueryDatabaseMetaDataIT.testRemarkColumn() fails if we don't have a cell for REMARKS.


---

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

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by churrodog <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(COLUMN_DEF), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SQL_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATA_TYPE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SQL_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SQL_DATETIME_SUB_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATETIME_SUB),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SQL_DATETIME_SUB), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl CHAR_OCTET_LENGTH_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, Bytes.toBytes(CHAR_OCTET_LENGTH), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl ORDINAL_POSITION_COLUMN = new PColumnImpl(PNameFactory.newName(ORDINAL_POSITION),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, ORDINAL_POSITION_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl IS_NULLABLE_COLUMN = new PColumnImpl(PNameFactory.newName(IS_NULLABLE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(IS_NULLABLE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_CATALOG_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_CATALOG),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_SCHEMA_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_SCHEMA),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_SCHEMA), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SCOPE_TABLE_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_TABLE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_TABLE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl SOURCE_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SOURCE_DATA_TYPE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SOURCE_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl IS_AUTOINCREMENT_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl ARRAY_SIZE_COLUMN = new PColumnImpl(PNameFactory.newName(ARRAY_SIZE),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, ARRAY_SIZE_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl COLUMN_FAMILY_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_FAMILY),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, COLUMN_FAMILY_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl TYPE_ID_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, Bytes.toBytes(TYPE_ID), HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl VIEW_CONSTANT_COLUMN = new PColumnImpl(PNameFactory.newName(VIEW_CONSTANT),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PVarbinary.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, VIEW_CONSTANT_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl MULTI_TENANT_COLUMN = new PColumnImpl(PNameFactory.newName(MULTI_TENANT),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PBoolean.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, MULTI_TENANT_BYTES, HConstants.LATEST_TIMESTAMP);
    +	private static final PColumnImpl KEY_SEQ_COLUMN = new PColumnImpl(PNameFactory.newName(KEY_SEQ),
    +			PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
    +			0, null, false, null, false, false, KEY_SEQ_BYTES, HConstants.LATEST_TIMESTAMP);
    +	
    +	private static final List<PColumnImpl> PK_DATUM_LIST = Lists.newArrayList(TENANT_ID_COLUMN, TABLE_SCHEM_COLUMN, TABLE_NAME_COLUMN, COLUMN_NAME_COLUMN);
    +	
    +	private static final RowProjector GET_COLUMNS_ROW_PROJECTOR = new RowProjector(
    +			Arrays.<ColumnProjector> asList(
    +					new ExpressionProjector(TABLE_CAT, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TENANT_ID_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 0)), false),
    +					new ExpressionProjector(TABLE_SCHEM, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TABLE_SCHEM_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 1)), false),
    +					new ExpressionProjector(TABLE_NAME, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(TABLE_NAME_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 2)), false),
    +					new ExpressionProjector(COLUMN_NAME, SYSTEM_CATALOG,
    +							new RowKeyColumnExpression(COLUMN_NAME_COLUMN,
    +									new RowKeyValueAccessor(PK_DATUM_LIST, 3)), false),
    +					new ExpressionProjector(DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(TYPE_NAME, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(TYPE_NAME_COLUMN), false),
    +					new ExpressionProjector(COLUMN_SIZE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_SIZE_COLUMN), false),
    +					new ExpressionProjector(BUFFER_LENGTH, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(BUFFER_LENGTH_COLUMN), false),
    +					new ExpressionProjector(DECIMAL_DIGITS, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(DECIMAL_DIGITS_COLUMN), false),
    +					new ExpressionProjector(NUM_PREC_RADIX, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(NUM_PREC_RADIX_COLUMN), false),
    +					new ExpressionProjector(NULLABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(NULLABLE_COLUMN), false),
    +					new ExpressionProjector(REMARKS, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(REMARKS_COLUMN), false),
    +					new ExpressionProjector(COLUMN_DEF, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_DEF_COLUMN), false),
    +					new ExpressionProjector(SQL_DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SQL_DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(SQL_DATETIME_SUB, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SQL_DATETIME_SUB_COLUMN), false),
    +					new ExpressionProjector(CHAR_OCTET_LENGTH, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(CHAR_OCTET_LENGTH_COLUMN), false),
    +					new ExpressionProjector(ORDINAL_POSITION, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(ORDINAL_POSITION_COLUMN), false),
    +					new ExpressionProjector(IS_NULLABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(IS_NULLABLE_COLUMN), false),
    +					new ExpressionProjector(SCOPE_CATALOG, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_CATALOG_COLUMN), false),
    +					new ExpressionProjector(SCOPE_SCHEMA, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_SCHEMA_COLUMN), false),
    +					new ExpressionProjector(SCOPE_TABLE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SCOPE_TABLE_COLUMN), false),
    +					new ExpressionProjector(SOURCE_DATA_TYPE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(SOURCE_DATA_TYPE_COLUMN), false),
    +					new ExpressionProjector(IS_AUTOINCREMENT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(IS_AUTOINCREMENT_COLUMN), false),
    +					new ExpressionProjector(ARRAY_SIZE, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(ARRAY_SIZE_COLUMN), false),
    +					new ExpressionProjector(COLUMN_FAMILY, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(COLUMN_FAMILY_COLUMN), false),
    +					new ExpressionProjector(TYPE_ID, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(TYPE_ID_COLUMN), false),
    +					new ExpressionProjector(VIEW_CONSTANT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(VIEW_CONSTANT_COLUMN), false),
    +					new ExpressionProjector(MULTI_TENANT, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(MULTI_TENANT_COLUMN), false),
    +					new ExpressionProjector(KEY_SEQ, SYSTEM_CATALOG,
    +							new KeyValueColumnExpression(KEY_SEQ_COLUMN), false)
    +					), 0, true);
    +	
    +	private boolean match(String str, String pattern) throws SQLException {
    +		LiteralExpression strExpr = LiteralExpression.newConstant(str, PVarchar.INSTANCE, SortOrder.ASC);
    +		LiteralExpression patternExpr = LiteralExpression.newConstant(pattern, PVarchar.INSTANCE, SortOrder.ASC);
    +		List<Expression> children = Arrays.<Expression>asList(strExpr, patternExpr);
    +		LikeExpression likeExpr = StringBasedLikeExpression.create(children, LikeType.CASE_SENSITIVE);
    +		ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    +		boolean evaluated = likeExpr.evaluate(null, ptr);
    +		Boolean result = (Boolean)likeExpr.getDataType().toObject(ptr);
    +		if (evaluated) {
    +			return result;
    +		}
    +		return false;
    +	}
    +    
    +	@Override
         public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
                 throws SQLException {
    -        StringBuilder buf = new StringBuilder("select \n " +
    -                TENANT_ID + " " + TABLE_CAT + "," + // use this for tenant id
    -                TABLE_SCHEM + "," +
    -                TABLE_NAME + " ," +
    -                COLUMN_NAME + "," +
    -                ExternalSqlTypeIdFunction.NAME + "(" + DATA_TYPE + ") AS " + DATA_TYPE + "," +
    -                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + "," +
    -                COLUMN_SIZE + "," +
    -                BUFFER_LENGTH + "," +
    -                DECIMAL_DIGITS + "," +
    -                NUM_PREC_RADIX + "," +
    -                NULLABLE + "," +
    -                REMARKS + "," +
    -                COLUMN_DEF + "," +
    -                SQL_DATA_TYPE + "," +
    -                SQL_DATETIME_SUB + "," +
    -                CHAR_OCTET_LENGTH + "," +
    -                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + ORDINAL_POSITION + "-1 ELSE " + ORDINAL_POSITION + " END AS " + ORDINAL_POSITION + "," +
    -                "CASE " + NULLABLE + " WHEN " + DatabaseMetaData.attributeNoNulls +  " THEN '" + Boolean.FALSE.toString() + "' WHEN " + DatabaseMetaData.attributeNullable + " THEN '" + Boolean.TRUE.toString() + "' END AS " + IS_NULLABLE + "," +
    -                SCOPE_CATALOG + "," +
    -                SCOPE_SCHEMA + "," +
    -                SCOPE_TABLE + "," +
    -                SOURCE_DATA_TYPE + "," +
    -                IS_AUTOINCREMENT + "," +
    -                ARRAY_SIZE + "," +
    -                COLUMN_FAMILY + "," +
    -                DATA_TYPE + " " + TYPE_ID + "," +// raw type id for potential internal consumption
    -                VIEW_CONSTANT + "," +
    -                MULTI_TENANT + "," +
    -                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + KEY_SEQ + "-1 ELSE " + KEY_SEQ + " END AS " + KEY_SEQ +
    -                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS + "(" + TENANT_POS_SHIFT + " BOOLEAN)");
    -        StringBuilder where = new StringBuilder();
    -        addTenantIdFilter(where, catalog);
    -        if (schemaPattern != null) {
    -            appendConjunction(where);
    -            where.append(TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'" ));
    -        }
    -        if (tableNamePattern != null && tableNamePattern.length() > 0) {
    -            appendConjunction(where);
    -            where.append(TABLE_NAME + " like '" + StringUtil.escapeStringConstant(tableNamePattern) + "'" );
    -        }
    -        // Allow a "." in columnNamePattern for column family match
    -        String colPattern = null;
    -        if (columnNamePattern != null && columnNamePattern.length() > 0) {
    +		boolean isTenantSpecificConnection = connection.getTenantId() != null;
    +    	List<Tuple> tuples = Lists.newArrayListWithExpectedSize(10);
    +    	ResultSet rs = getTables(catalog, schemaPattern, tableNamePattern, null);
    +    	while (rs.next()) {
    +    		String schemaName = rs.getString(TABLE_SCHEM);
    +    		boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
    +    		// Allow a "." in columnNamePattern for column family match
    +            String colPattern = null;
                 String cfPattern = null;
    -            int index = columnNamePattern.indexOf('.');
    -            if (index <= 0) {
    -                colPattern = columnNamePattern;
    -            } else {
    -                cfPattern = columnNamePattern.substring(0, index);
    -                if (columnNamePattern.length() > index+1) {
    -                    colPattern = columnNamePattern.substring(index+1);
    +            if (columnNamePattern != null && columnNamePattern.length() > 0) {
    +                int index = columnNamePattern.indexOf('.');
    +                if (index <= 0) {
    +                    colPattern = columnNamePattern;
    +                } else {
    +                    cfPattern = columnNamePattern.substring(0, index);
    +                    if (columnNamePattern.length() > index+1) {
    +                        colPattern = columnNamePattern.substring(index+1);
    +                    }
                     }
                 }
    -            if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
    -                // Will pick up only KV columns
    -                appendConjunction(where);
    -                where.append(COLUMN_FAMILY + " like '" + StringUtil.escapeStringConstant(cfPattern) + "'" );
    -            }
    -            if (colPattern != null && colPattern.length() > 0) {
    -                appendConjunction(where);
    -                where.append(COLUMN_NAME + " like '" + StringUtil.escapeStringConstant(colPattern) + "'" );
    -            }
    -        }
    -        if (colPattern == null || colPattern.length() == 0) {
    -            appendConjunction(where);
    -            where.append(COLUMN_NAME + " is not null" );
    -            appendConjunction(where);
    -            where.append(LINK_TYPE + " is null" );
    -        }
    -        boolean isTenantSpecificConnection = connection.getTenantId() != null;
    -        if (isTenantSpecificConnection) {
    -            buf.append(" where (" + where + ") OR ("
    -                    + COLUMN_FAMILY + " is null AND " +  COLUMN_NAME + " is null)");
    -        } else {
    -            buf.append(" where " + where);
    -        }
    -        buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME + "," + SYSTEM_CATALOG_ALIAS + "." + ORDINAL_POSITION);
    -
    -        Statement stmt;
    -        if (isTenantSpecificConnection) {
    -            stmt = connection.createStatement(new PhoenixStatementFactory() {
    -                @Override
    -                public PhoenixStatement newStatement(PhoenixConnection connection) {
    -                    return new PhoenixStatement(connection) {
    -                        @Override
    -                        public PhoenixResultSet newResultSet(ResultIterator iterator, RowProjector projector,
    -                                StatementContext context) throws SQLException {
    -                            return new PhoenixResultSet(new TenantColumnFilteringIterator(iterator, projector),
    -                                    projector, context);
    -                        }
    -                    };
    +			String tableName = rs.getString(TABLE_NAME);
    +    		String tenantId = rs.getString(TABLE_CAT);
    +    		String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
    +    		// if its a SYSTEM table we prevent the rpc to fetch the table so look it up in the cache
    +			PTable table = systemTable ? PhoenixRuntime.getTable(connection, fullTableName)
    +					: PhoenixRuntime.getTableNoCache(connection, fullTableName);
    --- End diff --
    
    Why not allow the cache to always be used?


---

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

Posted by twdsilva <gi...@git.apache.org>.
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<PTable, MetaDataProtos.MutationCode> 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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<MetaDataResponse> 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<Mutation> 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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<PColumn> 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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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<MetaDataResponse> 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<Mutation> 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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<PColumn> 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 ...

Posted by twdsilva <gi...@git.apache.org>.
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<PColumn> 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));
    +				viewColumnConstantsMatched[column.getPosition()]=true;
    +            }
    +        	// If view is not updatable, viewColumnConstants should be empty. We will still
    +            // inherit our parent viewConstants, but we have no additional ones.
    +        	else if(isViewReferenced ){
    +        		result.add(new PColumnImpl(column, column.getViewConstant(), isViewReferenced));
    +        	}
    +        	else {
    +                result.add(column);
    +            }
    +        }
    +		// ensure that node of the columns in the view where statement were
    +		// dropped in any of this views ancestors
    +//		for (int i = 0; i < viewColumnConstantsMatched.length; ++i) {
    --- End diff --
    
    I removed this code, its not needed.


---

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

Posted by churrodog <gi...@git.apache.org>.
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<String> 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<String> 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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by churrodog <gi...@git.apache.org>.
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<PColumn> 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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by twdsilva <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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 ...

Posted by JamesRTaylor <gi...@git.apache.org>.
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<MetaDataResponse> 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<Mutation> 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.


---