From dev-return-52952-archive-asf-public=cust-asf.ponee.io@phoenix.apache.org Wed Jul 11 19:24:01 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 5DAE118062A for ; Wed, 11 Jul 2018 19:24:00 +0200 (CEST) Received: (qmail 96452 invoked by uid 500); 11 Jul 2018 17:23:59 -0000 Mailing-List: contact dev-help@phoenix.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.apache.org Delivered-To: mailing list dev@phoenix.apache.org Received: (qmail 96441 invoked by uid 99); 11 Jul 2018 17:23:58 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 11 Jul 2018 17:23:58 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 75543DFC42; Wed, 11 Jul 2018 17:23:58 +0000 (UTC) From: twdsilva To: dev@phoenix.apache.org Reply-To: dev@phoenix.apache.org References: In-Reply-To: Subject: [GitHub] phoenix pull request #303: PHOENIX-3534 Support multi region SYSTEM.CATALOG ... Content-Type: text/plain Message-Id: <20180711172358.75543DFC42@git1-us-west.apache.org> Date: Wed, 11 Jul 2018 17:23:58 +0000 (UTC) Github user twdsilva commented on a diff in the pull request: https://github.com/apache/phoenix/pull/303#discussion_r201776009 --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java --- @@ -586,48 +590,359 @@ public void getTable(RpcController controller, GetTableRequest request, builder.setMutationTime(minNonZerodisableIndexTimestamp - 1); } } - - if (table.getTimeStamp() != tableTimeStamp) { + // the PTable of views and indexes on views might get updated because a column is added to one of + // their parents (this won't change the timestamp) + if (table.getType()!=PTableType.TABLE || table.getTimeStamp() != tableTimeStamp) { builder.setTable(PTableImpl.toProto(table)); } done.run(builder.build()); - return; } catch (Throwable t) { logger.error("getTable failed", t); ProtobufUtil.setControllerException(controller, ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t)); } } + /** + * Used to add the columns present the ancestor hierarchy to the PTable of the given view or + * view index + * @param table PTable of the view or view index + * @param skipAddingIndexes if true the returned PTable won't include indexes + * @param skipAddingParentColumns if true the returned PTable won't include columns derived from + * ancestor tables + * @param lockedAncestorTable ancestor table table that is being mutated (as we won't be able to + * resolve this table as its locked) + */ + private Pair combineColumns(PTable table, long timestamp, + int clientVersion, boolean skipAddingIndexes, boolean skipAddingParentColumns, + PTable lockedAncestorTable) throws SQLException, IOException { + boolean hasIndexId = table.getViewIndexId() != null; + if (table.getType() != PTableType.VIEW && !hasIndexId) { + return new Pair(table, + MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS); + } + if (!skipAddingParentColumns) { + table = + addDerivedColumnsFromAncestors(table, timestamp, clientVersion, + lockedAncestorTable); + if (table==null) { + return new Pair(table, + MetaDataProtos.MutationCode.TABLE_NOT_FOUND); + } + // we need to resolve the indexes of views (to get ensure they also have all the columns + // derived from their ancestors) + if (!skipAddingIndexes && !table.getIndexes().isEmpty()) { + List indexes = Lists.newArrayListWithExpectedSize(table.getIndexes().size()); + for (PTable index : table.getIndexes()) { + byte[] tenantIdBytes = + index.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY + : index.getTenantId().getBytes(); + PTable latestIndex = + doGetTable(tenantIdBytes, index.getSchemaName().getBytes(), + index.getTableName().getBytes(), timestamp, null, clientVersion, true, + false, lockedAncestorTable); + if (latestIndex == null) { + throw new TableNotFoundException( + "Could not find index table while combining columns " + + index.getTableName().getString() + " with tenant id " + + index.getTenantId()); + } + indexes.add(latestIndex); + } + table = PTableImpl.makePTable(table, table.getTimeStamp(), indexes); + } + } + + MetaDataProtos.MutationCode mutationCode = + table != null ? MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS + : MetaDataProtos.MutationCode.TABLE_NOT_FOUND; + return new Pair(table, mutationCode); + } + + + private PTable addDerivedColumnsFromAncestors(PTable table, long timestamp, + int clientVersion, PTable lockedAncestorTable) throws IOException, SQLException, TableNotFoundException { + // combine columns for view and view indexes + byte[] tenantId = + table.getTenantId() != null ? table.getTenantId().getBytes() + : 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 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 allColumns = Lists.newArrayList(); + List excludedColumns = Lists.newArrayList(); + // add my own columns first in reverse order + List 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. ---