You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by churrodog <gi...@git.apache.org> on 2017/05/15 21:14:55 UTC

[GitHub] phoenix pull request #248: Phoenix 3534

GitHub user churrodog opened a pull request:

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

    Phoenix 3534

    I'll squash these down after a review 

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

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

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

    https://github.com/apache/phoenix/pull/248.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 #248
    
----
commit a7712e3977830ffe715a2caaa577eb5cdb071e90
Author: rgidwani <rg...@salesforce.com>
Date:   2017-03-17T19:21:56Z

    Starting work on splittable System.Catalog

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

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

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

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

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

    Create table and read views work now

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

    Fixed the test - moving on to add drop columns

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

    getting tests and add column to work

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

    Figuring out the delete logic and refactoring the old tests

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

    Alter view drop column works!

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

    Drop Cascade and create check completed, need to test

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

    Drop cascade seems to work

commit 86a2633f57ec541330e56fd9c3a5da20a9cce165
Author: rgidwani <rg...@salesforce.com>
Date:   2017-05-15T21:13:30Z

    First pass at multi-region System.CATALOG

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix issue #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248
  
    This is looking really good, @churrodog. For testing purposes, I think it'd be really good if we split the SYSTEM.CATALOG table at points that'll force it to span multiple regions during the running of tests like AlterTableWithViewsIT, TenantSpecificTablesDDLIT, AlterTableIT, QueryDatabaseMetaDataIT, and maybe more.
    
    Any full table scans let in the CREATE or DROP code path?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix issue #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248
  
    @twdsilva would you mind taking a look as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r116626026
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java ---
    @@ -159,15 +199,6 @@ public SortOrder getSortOrder() {
         public String toString() {
             return (familyName == null ? "" : familyName.toString() + QueryConstants.NAME_SEPARATOR) + name.toString();
         }
    -    
    --- End diff --
    
    Should this be removed and if so why?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r116626201
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -381,7 +383,8 @@
                 IS_VIEW_REFERENCED_KV,
                 COLUMN_DEF_KV,
                 IS_ROW_TIMESTAMP_KV,
    -            COLUMN_QUALIFIER_KV
    +            COLUMN_QUALIFIER_KV,
    +            DROPPED_COLUMN_KV
    --- End diff --
    
    I don't think you need this DROPPED_COLUMN_KV since you're not reading this when the column is read (but instead when the linking rows are read). If that's right, let's remove it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r116659570
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -535,19 +539,115 @@ public void getTable(RpcController controller, GetTableRequest request,
                     // Subtract one because we add one due to timestamp granularity in Windows
                     builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
                 }
    -
    +            if (PTableType.VIEW.equals(table.getType())) {
    +                table = combineColumns(table, tenantId, schemaName, tableName, request.getClientTimestamp());
    +            }
                 if (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));
             }
         }
     
    +    private PTable combineColumns(PTable table, byte[] tenantId, byte[] schemaName, byte[] tableName, long timestamp)
    +        throws SQLException, IOException {
    +        // 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<byte[]> listOBytes = Lists.newArrayList();
    +        TableViewFinderResult viewFinderResult = new TableViewFinderResult();
    +        findAllParentViews(tenantId, schemaName, tableName, viewFinderResult);
    +        for (Result aResult : viewFinderResult.getResults()) {
    +            byte[][] rowViewKeyMetaData = new byte[5][];
    +            getVarChars(aResult.getRow(), 5, rowViewKeyMetaData);
    +            byte[] resultTenantId = rowViewKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
    +            byte[] resultSchema = SchemaUtil.getSchemaNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
    +            byte[] resultTable = SchemaUtil.getTableNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
    +            byte[] rowKeyInQuestion = SchemaUtil.getTableKey(resultTenantId, resultSchema, resultTable);
    +            listOBytes.add(rowKeyInQuestion);
    +        }
    +        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 {
    +                allColumns.add(pColumn);
    +            }
    +
    +        }
    +        // now go up from child to parent all the way to the base table:
    +        for (int i = 0; i < listOBytes.size(); i++) {
    +            byte[] tableInQuestion = listOBytes.get(i);
    +            PTable pTable = this.doGetTable(tableInQuestion, timestamp);
    +            if (pTable == null) {
    +                throw new TableNotFoundException("fill in with valuable info");
    --- End diff --
    
    fix the exception message


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r117566388
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java ---
    @@ -107,12 +107,14 @@ public void testAddNewColumnsToBaseTableWithViews() throws Exception {
                 assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
                 
                 viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
    -            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
    +            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "VIEW_COL1", "VIEW_COL2");
    --- End diff --
    
    Sure we can do that, I think we will need to modify a ton of tests to make this happen.  Its not only done here. 
    
    Another thing is that there are a few tests where the assumptions of how the system works has changed.  A few examples are tests like this: 
    
    ViewIT.testDisallowDropOfColumnOnParentTable()
    ViewIT.testDisallowDropOfReferencedColumn()
    QueryCompiler.testDuplicatePKColumn() 
    (this one because we have to support both the diff based approach and the current approach for our migration step, we remove duplicate columns on read, thus we don't prevent this from happening). 
    
    There are quite a few more, we can maybe discuss if these tests are no longer valid or how we can change them to make them meaningful. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r116659785
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java ---
    @@ -0,0 +1,241 @@
    +/**
    + * 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 com.google.common.collect.LinkedHashMultimap;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.ResultScanner;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.client.Table;
    +import org.apache.hadoop.hbase.filter.CompareFilter;
    +import org.apache.hadoop.hbase.filter.FilterList;
    +import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.tuple.ResultTuple;
    +import org.apache.phoenix.util.ByteUtil;
    +import org.apache.phoenix.util.SchemaUtil;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
    +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES;
    +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
    +import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
    +import static org.apache.phoenix.util.SchemaUtil.getVarChars;
    +
    +class ViewFinder {
    +
    +    static TableViewFinderResult findBaseTable(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
    +        throws IOException {
    +        return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.PHYSICAL_TABLE,
    +            HConstants.LATEST_TIMESTAMP);
    +    }
    +
    +    static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
    +        PTable.LinkType linkType, TableViewFinderResult result) throws IOException {
    +        findAllRelatives(systemCatalog, tenantId, schema, table, linkType, HConstants.LATEST_TIMESTAMP, result);
    +    }
    +
    +    static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
    +        PTable.LinkType linkType, long timestamp, TableViewFinderResult result) throws IOException {
    +        TableViewFinderResult currentResult =
    +            findRelatedViews(systemCatalog, tenantId, schema, table, linkType, timestamp);
    +        result.addResult(currentResult);
    +        for (Result viewResult : currentResult.getResults()) {
    +            byte[][] rowViewKeyMetaData = new byte[5][];
    +            getVarChars(viewResult.getRow(), 5, rowViewKeyMetaData);
    +            byte[] viewtenantId = rowViewKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
    +            byte[] viewSchema =
    +                SchemaUtil.getSchemaNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX])
    +                    .getBytes();
    +            byte[] viewTable =
    +                SchemaUtil.getTableNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX])
    +                    .getBytes();
    +            findAllRelatives(systemCatalog, viewtenantId, viewSchema, viewTable, linkType, timestamp, result);
    +        }
    +    }
    +
    +    static TableViewFinderResult findRelatedViews(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
    +        PTable.LinkType linkType, long timestamp) throws IOException {
    +        Scan scan = new Scan();
    +        byte[] startRow = SchemaUtil.getTableKey(tenantId, schema, table);
    +        byte[] stopRow = ByteUtil.nextKey(startRow);
    +        scan.setStartRow(startRow);
    +        scan.setStopRow(stopRow);
    +        scan.setTimeRange(0, timestamp);
    +        SingleColumnValueFilter linkFilter =
    +            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
    +                linkType.getSerializedValueAsByteArray());
    +        linkFilter.setFilterIfMissing(true);
    +        scan.setFilter(linkFilter);
    +        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
    +        scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
    +        List<Result> results = Lists.newArrayList();
    +        ResultScanner scanner = systemCatalog.getScanner(scan);
    +        try {
    +            for (Result result = scanner.next(); (result != null); result = scanner.next()) {
    +                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
    +                ResultTuple resultTuple = new ResultTuple(result);
    +                resultTuple.getKey(ptr);
    +                results.add(result);
    +            }
    +            return new TableViewFinderResult(results);
    +        } finally {
    +            scanner.close();
    +        }
    +    }
    +
    +    static Graph<TableInfo> findOrphans(Table systemCatalog, long timestamp) throws IOException {
    --- End diff --
    
    is this method used somewhere?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r116625600
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java ---
    @@ -107,12 +107,14 @@ public void testAddNewColumnsToBaseTableWithViews() throws Exception {
                 assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
                 
                 viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
    -            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
    +            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "VIEW_COL1", "VIEW_COL2");
    --- End diff --
    
    Are these test changes due to only storing diffs now? Would be nice to keep the check for "this is what the combined table should look like" in addition to checking that only the diffs are actually stored. WDYT?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix issue #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248
  
    I'll check it out, but in the meantime would you mind amending your commit message from "Phoenix 3534" to "PHOENIX-3534 Support multi region SYSTEM.CATALOG table" - that way comments on the pull will become JIRA comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix issue #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248
  
    Looks like there are still conflicts, @churrodog. Would you mind amending your commit message to "PHOENIX-3534 Support multi region SYSTEM.CATALOG table" so that the JIRA is linked to the PR?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r116625014
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java ---
    @@ -500,8 +510,20 @@ private void init(PName tenantId, PName schemaName, PName tableName, PTableType
                 allColumns = new PColumn[columns.size()];
                 pkColumns = Lists.newArrayListWithExpectedSize(columns.size());
             }
    -        for (PColumn column : columns) {
    -            allColumns[column.getPosition()] = column;
    +        // Must do this as with the new method of storing diffs, we just care about ordinal position
    +        // relative order and not the true ordinal value itself.
    +        List<PColumn> sortedColumns = Lists.newArrayList(columns);
    +        Collections.sort(sortedColumns, new Comparator<PColumn>() {
    +            @Override
    +            public int compare(PColumn o1, PColumn o2) {
    +                return Integer.valueOf(o1.getPosition()).compareTo(o2.getPosition());
    --- End diff --
    
    What will prevent overlapping ordinals between a base table and derived views?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r116659343
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java ---
    @@ -0,0 +1,50 @@
    +/**
    + * 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 com.google.common.collect.Lists;
    +import org.apache.hadoop.hbase.client.Result;
    +
    +import java.util.List;
    +
    +/**
    + * Certain operations, such as DROP TABLE are not allowed if there a table has child views. This class wraps the
    + * Results of a scanning the Phoenix Metadata for child views for a specific table and stores an additional flag for
    + * whether whether SYSTEM.CATALOG has split across multiple regions.
    --- End diff --
    
    modify this comment 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] phoenix pull request #248: Phoenix 3534

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

    https://github.com/apache/phoenix/pull/248#discussion_r116625769
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2324,96 +2488,97 @@ else if (pkCount <= COLUMN_NAME_INDEX
                                         if (ordinalPos >= newOrdinalPosition) {
                                             if (ordinalPos == existingOrdinalPos) {
                                                 /*
    -                                             * No need to update ordinal positions of columns beyond the existing column's 
    +                                             * No need to update ordinal positions of columns beyond the existing column's
                                                  * old ordinal position.
                                                  */
                                                 break;
                                             }
                                             // increment ordinal position of columns occurring after this column by 1
                                             int updatedPos = ordinalPos + 1;
                                             ordinalPositionList.addColumn(getColumnKey(viewKey, col), updatedPos);
    -                                    } 
    +                                    }
                                     }
    -                            } 
    +                            }
                             } else {
                                 if (existingOrdinalPos != newOrdinalPosition) {
                                     ordinalPositionList.addColumn(columnKey, newOrdinalPosition);
                                 }
                             }
                             columnsAddedToBaseTable++;
                         }
    -                } else {
    -                    // The column doesn't exist in the view.
    -                    Put viewColumnPut = new Put(columnKey, clientTimeStamp);
    -                    for (Cell cell : baseTableColumnPut.getFamilyCellMap().values().iterator().next()) {
    -                        viewColumnPut.add(CellUtil.createCell(columnKey, CellUtil.cloneFamily(cell),
    -                                CellUtil.cloneQualifier(cell), cell.getTimestamp(), cell.getTypeByte(),
    -                                CellUtil.cloneValue(cell)));
    -                    }
    -                    if (isDivergedView(view)) {
    -                        if (isPkCol) {
    -                            /* 
    -                             * Only pk cols of the base table are added to the diverged views. These pk 
    -                             * cols are added at the end.
    -                             */
    -                            int lastOrdinalPos = getOrdinalPosition(view, view.getColumns().get(numCols - 1));
    -                            int newPosition = ++lastOrdinalPos;
    -                            byte[] ptr = new byte[PInteger.INSTANCE.getByteSize()];
    -                            PInteger.INSTANCE.getCodec().encodeInt(newPosition, ptr, 0);
    -                            viewColumnPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
    -                                    PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES, clientTimeStamp, ptr);
    -                            mutationsForAddingColumnsToViews.add(viewColumnPut);
    -                        } else {
    -                            continue; // move on to the next column
    -                        }
    -                    } else {
    -                        int newOrdinalPosition = p.ordinalPosition;
    -                        /*
    -                         * For a non-diverged view, we need to make sure that the base table column
    -                         * is added at the right position.
    -                         */
    -                        if (ordinalPositionList.size() == 0) {
    -                            ordinalPositionList.setOffset(newOrdinalPosition);
    -                            ordinalPositionList.addColumn(columnKey, newOrdinalPosition);
    -                            for (PColumn col : view.getColumns()) {
    -                                int ordinalPos = getOrdinalPosition(view, col);
    -                                if (ordinalPos >= newOrdinalPosition) {
    -                                    // increment ordinal position of columns by 1
    -                                    int updatedPos = ordinalPos + 1;
    -                                    ordinalPositionList.addColumn(getColumnKey(viewKey, col), updatedPos);
    -                                } 
    -                            }
    -                        } else {
    -                            ordinalPositionList.addColumn(columnKey, newOrdinalPosition);
    -                        }
    -                        mutationsForAddingColumnsToViews.add(viewColumnPut);
    -                    }
    -                    if (isPkCol) {
    -                        deltaNumPkColsSoFar++;
    -                        // Set the key sequence for the pk column to be added
    -                        short currentKeySeq = SchemaUtil.getMaxKeySeq(view);
    -                        short newKeySeq = (short)(currentKeySeq + deltaNumPkColsSoFar);
    -                        byte[] keySeqBytes = new byte[PSmallint.INSTANCE.getByteSize()];
    -                        PSmallint.INSTANCE.getCodec().encodeShort(newKeySeq, keySeqBytes, 0);
    -                        viewColumnPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
    -                                PhoenixDatabaseMetaData.KEY_SEQ_BYTES, keySeqBytes);
    -                        addMutationsForAddingPkColsToViewIndexes(mutationsForAddingColumnsToViews, clientTimeStamp, view,
    -                                deltaNumPkColsSoFar, columnName, viewColumnPut);
    -                    }
    -                    columnsAddedToView++;
    -                    columnsAddedToBaseTable++;
                     }
    +//                else {
    --- End diff --
    
    You'll likely get to this later, but let's remove the commented out code when you're ready.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---