private PTable getTableFromCells()

in phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java [1142:1707]


    private PTable getTableFromCells(List<Cell> tableCellList, List<List<Cell>> allColumnCellList,
                                     long clientTimeStamp, int clientVersion, PTable oldTable)
        throws IOException, SQLException {
        Cell[] tableKeyValues = new Cell[TABLE_KV_COLUMNS.size()];
        Cell[] colKeyValues = new Cell[COLUMN_KV_COLUMNS.size()];

        // Create PTable based on KeyValues from scan
        Cell keyValue = tableCellList.get(0);
        byte[] keyBuffer = keyValue.getRowArray();
        int keyLength = keyValue.getRowLength();
        int keyOffset = keyValue.getRowOffset();
        PName tenantId = newPName(keyBuffer, keyOffset, keyLength);
        int tenantIdLength = (tenantId == null) ? 0 : tenantId.getBytes().length;
        if (tenantIdLength == 0) {
            tenantId = null;
        }
        PName schemaName = newPName(keyBuffer, keyOffset + tenantIdLength + 1, keyLength);
        int schemaNameLength = schemaName.getBytes().length;
        int tableNameLength = keyLength - schemaNameLength - 1 - tenantIdLength - 1;
        byte[] tableNameBytes = new byte[tableNameLength];
        System.arraycopy(keyBuffer, keyOffset + schemaNameLength + 1 + tenantIdLength + 1,
                tableNameBytes, 0, tableNameLength);
        PName tableName = PNameFactory.newName(tableNameBytes);

        int offset = tenantIdLength + schemaNameLength + tableNameLength + 3;
        // This will prevent the client from continually looking for the current
        // table when we know that there will never be one since we disallow updates
        // unless the table is the latest

        long timeStamp = keyValue.getTimestamp();
        PTableImpl.Builder builder = null;
        if (oldTable != null) {
            builder = PTableImpl.builderFromExisting(oldTable);
            List<PColumn> columns = oldTable.getColumns();
            if (oldTable.getBucketNum() != null && oldTable.getBucketNum() > 0) {
                //if it's salted, skip the salt column -- it will get added back during
                //the build process
                columns = columns.stream().skip(1).collect(Collectors.toList());
            }
            builder.setColumns(columns);
        } else {
            builder = new PTableImpl.Builder();
        }
        builder.setTenantId(tenantId);
        builder.setSchemaName(schemaName);
        builder.setTableName(tableName);

        int i = 0;
        int j = 0;
        while (i < tableCellList.size() && j < TABLE_KV_COLUMNS.size()) {
            Cell kv = tableCellList.get(i);
            Cell searchKv = TABLE_KV_COLUMNS.get(j);
            int cmp =
                    Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
                            kv.getQualifierLength(), searchKv.getQualifierArray(),
                            searchKv.getQualifierOffset(), searchKv.getQualifierLength());
            if (cmp == 0) {
                timeStamp = Math.max(timeStamp, kv.getTimestamp()); // Find max timestamp of table
                // header row
                tableKeyValues[j++] = kv;
                i++;
            } else if (cmp > 0) {
                timeStamp = Math.max(timeStamp, kv.getTimestamp());
                tableKeyValues[j++] = null;
            } else {
                i++; // shouldn't happen - means unexpected KV in system table header row
            }
        }
        // TABLE_TYPE, TABLE_SEQ_NUM and COLUMN_COUNT are required.
        if (tableKeyValues[TABLE_TYPE_INDEX] == null || tableKeyValues[TABLE_SEQ_NUM_INDEX] == null
                || tableKeyValues[COLUMN_COUNT_INDEX] == null) {
            // since we allow SYSTEM.CATALOG to split in certain cases there might be child links or
            // other metadata rows that are invalid and should be ignored
            Cell cell = tableCellList.get(0);
            LOGGER.error("Found invalid metadata rows for rowkey " +
                    Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
            return null;
        }

        Cell tableTypeKv = tableKeyValues[TABLE_TYPE_INDEX];
        PTableType tableType =
                PTableType
                        .fromSerializedValue(tableTypeKv.getValueArray()[tableTypeKv.getValueOffset()]);
        builder.setType(tableType);

        Cell tableSeqNumKv = tableKeyValues[TABLE_SEQ_NUM_INDEX];
        long tableSeqNum =
                PLong.INSTANCE.getCodec().decodeLong(tableSeqNumKv.getValueArray(),
                        tableSeqNumKv.getValueOffset(), SortOrder.getDefault());
        builder.setSequenceNumber(tableSeqNum);

        Cell columnCountKv = tableKeyValues[COLUMN_COUNT_INDEX];
        int columnCount =
                PInteger.INSTANCE.getCodec().decodeInt(columnCountKv.getValueArray(),
                        columnCountKv.getValueOffset(), SortOrder.getDefault());

        Cell pkNameKv = tableKeyValues[PK_NAME_INDEX];
        PName pkName =
                pkNameKv != null ? newPName(pkNameKv.getValueArray(), pkNameKv.getValueOffset(),
                        pkNameKv.getValueLength()) : null;
        builder.setPkName(pkName != null ? pkName : oldTable != null ? oldTable.getPKName() : null);

        Cell saltBucketNumKv = tableKeyValues[SALT_BUCKETS_INDEX];
        Integer saltBucketNum =
                saltBucketNumKv != null ? (Integer) PInteger.INSTANCE.getCodec().decodeInt(
                        saltBucketNumKv.getValueArray(), saltBucketNumKv.getValueOffset(), SortOrder.getDefault()) : null;
        if (saltBucketNum != null && saltBucketNum.intValue() == 0) {
            saltBucketNum = null; // Zero salt buckets means not salted
        }
        builder.setBucketNum(saltBucketNum != null ? saltBucketNum : oldTable != null ? oldTable.getBucketNum() : null);

        //data table name is used to find the parent table for indexes later
        Cell dataTableNameKv = tableKeyValues[DATA_TABLE_NAME_INDEX];
        PName dataTableName =
                dataTableNameKv != null ? newPName(dataTableNameKv.getValueArray(),
                        dataTableNameKv.getValueOffset(), dataTableNameKv.getValueLength()) : null;

        Cell physicalTableNameKv = tableKeyValues[PHYSICAL_TABLE_NAME_INDEX];
        PName physicalTableName =
                physicalTableNameKv != null ? newPName(physicalTableNameKv.getValueArray(),
                        physicalTableNameKv.getValueOffset(), physicalTableNameKv.getValueLength()) : null;
        builder.setPhysicalTableName(physicalTableName != null ? physicalTableName : oldTable != null ? oldTable.getPhysicalName(true) : null);

        Cell indexStateKv = tableKeyValues[INDEX_STATE_INDEX];
        PIndexState indexState =
                indexStateKv == null ? null : PIndexState.fromSerializedValue(indexStateKv
                        .getValueArray()[indexStateKv.getValueOffset()]);
        builder.setState(indexState != null ? indexState : oldTable != null ? oldTable.getIndexState() : null);

        Cell immutableRowsKv = tableKeyValues[IMMUTABLE_ROWS_INDEX];
        boolean isImmutableRows = immutableRowsKv != null && (Boolean) PBoolean.INSTANCE.toObject(
            immutableRowsKv.getValueArray(), immutableRowsKv.getValueOffset(),
            immutableRowsKv.getValueLength());
        builder.setImmutableRows(immutableRowsKv != null ? isImmutableRows :
            oldTable != null && oldTable.isImmutableRows());

        Cell defaultFamilyNameKv = tableKeyValues[DEFAULT_COLUMN_FAMILY_INDEX];
        PName defaultFamilyName = defaultFamilyNameKv != null ? newPName(defaultFamilyNameKv.getValueArray(), defaultFamilyNameKv.getValueOffset(), defaultFamilyNameKv.getValueLength()) : null;
        builder.setDefaultFamilyName(defaultFamilyName != null ? defaultFamilyName : oldTable != null ? oldTable.getDefaultFamilyName() : null);

        Cell viewStatementKv = tableKeyValues[VIEW_STATEMENT_INDEX];
        String viewStatement = viewStatementKv != null ? (String) PVarchar.INSTANCE.toObject(viewStatementKv.getValueArray(), viewStatementKv.getValueOffset(),
                viewStatementKv.getValueLength()) : null;
        builder.setViewStatement(viewStatement != null ? viewStatement : oldTable != null ? oldTable.getViewStatement() : null);

        Cell disableWALKv = tableKeyValues[DISABLE_WAL_INDEX];
        boolean disableWAL = disableWALKv == null ? PTable.DEFAULT_DISABLE_WAL : Boolean.TRUE.equals(
                PBoolean.INSTANCE.toObject(disableWALKv.getValueArray(), disableWALKv.getValueOffset(), disableWALKv.getValueLength()));
        builder.setDisableWAL(disableWALKv != null ? disableWAL :
            oldTable != null && oldTable.isWALDisabled());

        Cell multiTenantKv = tableKeyValues[MULTI_TENANT_INDEX];
        boolean multiTenant = multiTenantKv != null && Boolean.TRUE.equals(
            PBoolean.INSTANCE.toObject(multiTenantKv.getValueArray(),
                multiTenantKv.getValueOffset(), multiTenantKv.getValueLength()));
        builder.setMultiTenant(multiTenantKv != null ? multiTenant :
            oldTable != null && oldTable.isMultiTenant());

        Cell storeNullsKv = tableKeyValues[STORE_NULLS_INDEX];
        boolean storeNulls = storeNullsKv != null && Boolean.TRUE.equals(
            PBoolean.INSTANCE.toObject(storeNullsKv.getValueArray(), storeNullsKv.getValueOffset(),
                storeNullsKv.getValueLength()));
        builder.setStoreNulls(storeNullsKv != null ? storeNulls :
            oldTable != null && oldTable.getStoreNulls());

        Cell transactionalKv = tableKeyValues[TRANSACTIONAL_INDEX];
        Cell transactionProviderKv = tableKeyValues[TRANSACTION_PROVIDER_INDEX];
        TransactionFactory.Provider transactionProvider = null;
        if (transactionProviderKv == null) {
            if (transactionalKv != null && Boolean.TRUE.equals(
                    PBoolean.INSTANCE.toObject(
                            transactionalKv.getValueArray(),
                            transactionalKv.getValueOffset(),
                            transactionalKv.getValueLength()))) {
                // For backward compat, prior to client setting TRANSACTION_PROVIDER
                transactionProvider = TransactionFactory.Provider.NOTAVAILABLE;
            }
        } else {
            transactionProvider = TransactionFactory.Provider.fromCode(
                    PTinyint.INSTANCE.getCodec().decodeByte(
                            transactionProviderKv.getValueArray(),
                            transactionProviderKv.getValueOffset(),
                            SortOrder.getDefault()));
        }
        builder.setTransactionProvider(transactionProviderKv != null || transactionalKv != null
            ? transactionProvider : oldTable != null ? oldTable.getTransactionProvider() : null);

        Cell viewTypeKv = tableKeyValues[VIEW_TYPE_INDEX];
        ViewType viewType = viewTypeKv == null ? null : ViewType.fromSerializedValue(viewTypeKv.getValueArray()[viewTypeKv.getValueOffset()]);
        builder.setViewType(viewType != null ? viewType : oldTable != null ? oldTable.getViewType() : null);

        PDataType viewIndexIdType = oldTable != null ? oldTable.getviewIndexIdType() :
            getViewIndexIdType(tableKeyValues);
        builder.setViewIndexIdType(viewIndexIdType);

        Long viewIndexId = getViewIndexId(tableKeyValues, viewIndexIdType);
        builder.setViewIndexId(viewIndexId != null ? viewIndexId : oldTable != null ? oldTable.getViewIndexId() : null);

        Cell indexTypeKv = tableKeyValues[INDEX_TYPE_INDEX];
        IndexType indexType = indexTypeKv == null ? null : IndexType.fromSerializedValue(indexTypeKv.getValueArray()[indexTypeKv.getValueOffset()]);
        builder.setIndexType(indexType != null ? indexType : oldTable != null ? oldTable.getIndexType() : null);

        Cell baseColumnCountKv = tableKeyValues[BASE_COLUMN_COUNT_INDEX];
        int baseColumnCount = baseColumnCountKv == null ? 0 : PInteger.INSTANCE.getCodec().decodeInt(baseColumnCountKv.getValueArray(),
                baseColumnCountKv.getValueOffset(), SortOrder.getDefault());
        builder.setBaseColumnCount(baseColumnCountKv != null ? baseColumnCount : oldTable != null ? oldTable.getBaseColumnCount() : 0);

        Cell rowKeyOrderOptimizableKv = tableKeyValues[ROW_KEY_ORDER_OPTIMIZABLE_INDEX];
        boolean rowKeyOrderOptimizable = rowKeyOrderOptimizableKv != null && Boolean.TRUE.equals(
            PBoolean.INSTANCE.toObject(rowKeyOrderOptimizableKv.getValueArray(),
                rowKeyOrderOptimizableKv.getValueOffset(),
                rowKeyOrderOptimizableKv.getValueLength()));
        builder.setRowKeyOrderOptimizable(rowKeyOrderOptimizableKv != null ? rowKeyOrderOptimizable :
            oldTable != null && oldTable.rowKeyOrderOptimizable());

        Cell updateCacheFrequencyKv = tableKeyValues[UPDATE_CACHE_FREQUENCY_INDEX];
        long updateCacheFrequency = updateCacheFrequencyKv == null ? 0 :
                PLong.INSTANCE.getCodec().decodeLong(updateCacheFrequencyKv.getValueArray(),
                        updateCacheFrequencyKv.getValueOffset(), SortOrder.getDefault());
        builder.setUpdateCacheFrequency(updateCacheFrequencyKv != null ? updateCacheFrequency : oldTable != null ? oldTable.getUpdateCacheFrequency() : 0);

        // Check the cell tag to see whether the view has modified this property
        final byte[] tagUpdateCacheFreq = (updateCacheFrequencyKv == null) ?
         HConstants.EMPTY_BYTE_ARRAY :
                TagUtil.concatTags(HConstants.EMPTY_BYTE_ARRAY, updateCacheFrequencyKv);
        boolean viewModifiedUpdateCacheFrequency = (PTableType.VIEW.equals(tableType)) &&
                Bytes.contains(tagUpdateCacheFreq, MetaDataEndpointImplConstants.VIEW_MODIFIED_PROPERTY_BYTES);
        builder.setViewModifiedUpdateCacheFrequency(!Bytes.equals(tagUpdateCacheFreq,
            HConstants.EMPTY_BYTE_ARRAY) ? viewModifiedUpdateCacheFrequency :
            oldTable != null && oldTable.hasViewModifiedUpdateCacheFrequency());

        Cell indexDisableTimestampKv = tableKeyValues[INDEX_DISABLE_TIMESTAMP];
        long indexDisableTimestamp = indexDisableTimestampKv == null ? 0L : PLong.INSTANCE.getCodec().decodeLong(indexDisableTimestampKv.getValueArray(),
                indexDisableTimestampKv.getValueOffset(), SortOrder.getDefault());
        builder.setIndexDisableTimestamp(indexDisableTimestampKv != null ?
            indexDisableTimestamp : oldTable != null ? oldTable.getIndexDisableTimestamp() : 0L);

        Cell isNamespaceMappedKv = tableKeyValues[IS_NAMESPACE_MAPPED_INDEX];
        boolean isNamespaceMapped = isNamespaceMappedKv != null && Boolean.TRUE.equals(
            PBoolean.INSTANCE.toObject(isNamespaceMappedKv.getValueArray(),
                isNamespaceMappedKv.getValueOffset(), isNamespaceMappedKv.getValueLength()));
        builder.setNamespaceMapped(isNamespaceMappedKv != null ? isNamespaceMapped :
            oldTable != null && oldTable.isNamespaceMapped());

        Cell autoPartitionSeqKv = tableKeyValues[AUTO_PARTITION_SEQ_INDEX];
        String autoPartitionSeq = autoPartitionSeqKv != null ? (String) PVarchar.INSTANCE.toObject(autoPartitionSeqKv.getValueArray(), autoPartitionSeqKv.getValueOffset(),
                autoPartitionSeqKv.getValueLength()) : null;
        builder.setAutoPartitionSeqName(autoPartitionSeq != null
            ? autoPartitionSeq : oldTable != null ? oldTable.getAutoPartitionSeqName() : null);

        Cell isAppendOnlySchemaKv = tableKeyValues[APPEND_ONLY_SCHEMA_INDEX];
        boolean isAppendOnlySchema = isAppendOnlySchemaKv != null && Boolean.TRUE.equals(
            PBoolean.INSTANCE.toObject(isAppendOnlySchemaKv.getValueArray(),
                isAppendOnlySchemaKv.getValueOffset(), isAppendOnlySchemaKv.getValueLength()));
        builder.setAppendOnlySchema(isAppendOnlySchemaKv != null ? isAppendOnlySchema :
            oldTable != null && oldTable.isAppendOnlySchema());

        Cell storageSchemeKv = tableKeyValues[STORAGE_SCHEME_INDEX];
        //TODO: change this once we start having other values for storage schemes
        ImmutableStorageScheme storageScheme = storageSchemeKv == null ? ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ImmutableStorageScheme
                .fromSerializedValue((byte) PTinyint.INSTANCE.toObject(storageSchemeKv.getValueArray(),
                        storageSchemeKv.getValueOffset(), storageSchemeKv.getValueLength()));
        builder.setImmutableStorageScheme(storageSchemeKv != null ? storageScheme :
            oldTable != null ? oldTable.getImmutableStorageScheme() : ImmutableStorageScheme.ONE_CELL_PER_COLUMN);

        Cell encodingSchemeKv = tableKeyValues[QUALIFIER_ENCODING_SCHEME_INDEX];
        QualifierEncodingScheme encodingScheme = encodingSchemeKv == null ? QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : QualifierEncodingScheme
                .fromSerializedValue((byte) PTinyint.INSTANCE.toObject(encodingSchemeKv.getValueArray(),
                        encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
        builder.setQualifierEncodingScheme(encodingSchemeKv != null ? encodingScheme :
            oldTable != null ? oldTable.getEncodingScheme() : QualifierEncodingScheme.NON_ENCODED_QUALIFIERS);

        Cell useStatsForParallelizationKv = tableKeyValues[USE_STATS_FOR_PARALLELIZATION_INDEX];
        Boolean useStatsForParallelization = useStatsForParallelizationKv == null ? null :
            Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(useStatsForParallelizationKv.getValueArray(), useStatsForParallelizationKv.getValueOffset(), useStatsForParallelizationKv.getValueLength()));
         builder.setUseStatsForParallelization(useStatsForParallelization != null ?
             useStatsForParallelization : oldTable != null ? oldTable.useStatsForParallelization() : null);
        Cell lastDDLTimestampKv = tableKeyValues[LAST_DDL_TIMESTAMP_INDEX];
        Long lastDDLTimestamp = lastDDLTimestampKv == null ?
           null : PLong.INSTANCE.getCodec().decodeLong(lastDDLTimestampKv.getValueArray(),
                lastDDLTimestampKv.getValueOffset(), SortOrder.getDefault());
        builder.setLastDDLTimestamp(lastDDLTimestampKv != null ? lastDDLTimestamp :
            oldTable != null ? oldTable.getLastDDLTimestamp() : timeStamp);

        Cell changeDetectionEnabledKv = tableKeyValues[CHANGE_DETECTION_ENABLED_INDEX];
        boolean isChangeDetectionEnabled = changeDetectionEnabledKv != null
            && Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(changeDetectionEnabledKv.getValueArray(),
            changeDetectionEnabledKv.getValueOffset(),
            changeDetectionEnabledKv.getValueLength()));
        builder.setIsChangeDetectionEnabled(changeDetectionEnabledKv != null ?
            isChangeDetectionEnabled : oldTable != null && oldTable.isChangeDetectionEnabled());

        Cell schemaVersionKv = tableKeyValues[SCHEMA_VERSION_INDEX];
        String schemaVersion = schemaVersionKv != null ? (String) PVarchar.INSTANCE.toObject(
                schemaVersionKv.getValueArray(), schemaVersionKv.getValueOffset(), schemaVersionKv.getValueLength())
                : null;
        builder.setSchemaVersion(schemaVersion != null ?
            schemaVersion : oldTable != null ? oldTable.getSchemaVersion() : null);

        Cell externalSchemaIdKv = tableKeyValues[EXTERNAL_SCHEMA_ID_INDEX];
        String externalSchemaId = externalSchemaIdKv != null ?
            (String) PVarchar.INSTANCE.toObject(externalSchemaIdKv.getValueArray(),
                externalSchemaIdKv.getValueOffset(), externalSchemaIdKv.getValueLength())
            : null;
        builder.setExternalSchemaId(externalSchemaId != null ? externalSchemaId :
            oldTable != null ? oldTable.getExternalSchemaId() : null);

        Cell streamingTopicNameKv = tableKeyValues[STREAMING_TOPIC_NAME_INDEX];
        String streamingTopicName = streamingTopicNameKv != null ?
            (String) PVarchar.INSTANCE.toObject(streamingTopicNameKv.getValueArray(),
                streamingTopicNameKv.getValueOffset(), streamingTopicNameKv.getValueLength())
            : null;
        builder.setStreamingTopicName(streamingTopicName != null ? streamingTopicName :
            oldTable != null ? oldTable.getStreamingTopicName() : null);

        Cell includeSpecKv = tableKeyValues[CDC_INCLUDE_INDEX];
        String includeSpec = includeSpecKv != null ?
                (String) PVarchar.INSTANCE.toObject(includeSpecKv.getValueArray(),
                        includeSpecKv.getValueOffset(), includeSpecKv.getValueLength())
                : null;
        builder.setCDCIncludeScopes(includeSpec != null ?
                CDCUtil.makeChangeScopeEnumsFromString(includeSpec) :
                oldTable != null ? oldTable.getCDCIncludeScopes() : null);

        Cell indexWhereKv = tableKeyValues[INDEX_WHERE_INDEX];
        String indexWhere = indexWhereKv != null
                ? (String) PVarchar.INSTANCE.toObject(indexWhereKv.getValueArray(),
                        indexWhereKv.getValueOffset(), indexWhereKv.getValueLength())
                : null;
        builder.setIndexWhere(indexWhere != null ? indexWhere
                : oldTable != null ? oldTable.getIndexWhere() : null);
        Cell ttlKv = tableKeyValues[TTL_INDEX];
        TTLExpression ttl = TTL_EXPRESSION_NOT_DEFINED;
        if (ttlKv != null) {
            String ttlStr = (String) PVarchar.INSTANCE.toObject(
                    ttlKv.getValueArray(),
                    ttlKv.getValueOffset(),
                    ttlKv.getValueLength());
            ttl = TTLExpressionFactory.create(ttlStr);
        }
        ttl = ttlKv != null ? ttl : oldTable != null
                ? oldTable.getTTLExpression() : TTL_EXPRESSION_NOT_DEFINED;
        if (tableType == VIEW && viewType != MAPPED && ttl.equals(TTL_EXPRESSION_NOT_DEFINED)) {
            //Scan SysCat to get TTL from Parent View/Table
            byte[] viewKey = SchemaUtil.getTableKey(tenantId == null ? null : tenantId.getBytes(),
                    schemaName == null ? null : schemaName.getBytes(), tableNameBytes);
            ttl = getTTLFromHierarchy(viewKey, clientTimeStamp, false);

            // TODO: Need to Update Cache for Alter Commands, can use PHOENIX-6883.
        }

        Cell rowKeyMatcherKv = tableKeyValues[ROW_KEY_MATCHER_INDEX];
        byte[] rowKeyMatcher = rowKeyMatcherKv != null
                ? CellUtil.cloneValue(rowKeyMatcherKv)
                : HConstants.EMPTY_BYTE_ARRAY;
        builder.setRowKeyMatcher(rowKeyMatcher != null ? rowKeyMatcher
                : oldTable != null ? oldTable.getRowKeyMatcher() : HConstants.EMPTY_BYTE_ARRAY);


        // Check the cell tag to see whether the view has modified this property
        final byte[] tagUseStatsForParallelization = (useStatsForParallelizationKv == null) ?
                HConstants.EMPTY_BYTE_ARRAY :
                TagUtil.concatTags(HConstants.EMPTY_BYTE_ARRAY, useStatsForParallelizationKv);
        boolean viewModifiedUseStatsForParallelization = (PTableType.VIEW.equals(tableType)) &&
                Bytes.contains(tagUseStatsForParallelization, MetaDataEndpointImplConstants.VIEW_MODIFIED_PROPERTY_BYTES);
        builder.setViewModifiedUseStatsForParallelization(viewModifiedUseStatsForParallelization ||
            (oldTable != null && oldTable.hasViewModifiedUseStatsForParallelization()));

        boolean setPhysicalName = false;
        List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
        List<PTable> indexes = Lists.newArrayList();
        List<PName> physicalTables = Lists.newArrayList();
        PName parentTableName = tableType == INDEX || tableType == CDC ? dataTableName : null;
        PName parentSchemaName = tableType == INDEX || tableType == CDC ? schemaName : null;
        PName parentLogicalName = null;
        EncodedCQCounter cqCounter = null;
        if (oldTable != null) {
            cqCounter = oldTable.getEncodedCQCounter();
        } else {
            cqCounter = (!EncodedColumnsUtil.usesEncodedColumnNames(encodingScheme) || tableType == PTableType.VIEW) ?
                PTable.EncodedCQCounter.NULL_COUNTER :
                new EncodedCQCounter();
        }

        if (timeStamp == HConstants.LATEST_TIMESTAMP) {
            timeStamp = lastDDLTimestamp != null ? lastDDLTimestamp : clientTimeStamp;
        }
        builder.setTimeStamp(timeStamp);


        PTable transformingNewTable = null;
        boolean isRegularView = (tableType == PTableType.VIEW && viewType != MAPPED);
        boolean isThisAViewIndex = false;
        for (List<Cell> columnCellList : allColumnCellList) {

            Cell colKv = columnCellList.get(LINK_TYPE_INDEX);
            int colKeyLength = colKv.getRowLength();

            PName colName = newPName(colKv.getRowArray(), colKv.getRowOffset() + offset, colKeyLength - offset);
            if (colName == null) {
                continue;
            }
            int colKeyOffset = offset + colName.getBytes().length + 1;
            PName famName = newPName(colKv.getRowArray(), colKv.getRowOffset() + colKeyOffset, colKeyLength - colKeyOffset);

            if (isQualifierCounterKV(colKv)) {
                if (famName != null) {
                    Integer value = PInteger.INSTANCE.getCodec().decodeInt(colKv.getValueArray(), colKv.getValueOffset(), SortOrder.ASC);
                    cqCounter.setValue(famName.getString(), value);
                }
            } else if (Bytes.compareTo(LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length, colKv.getQualifierArray(), colKv.getQualifierOffset(), colKv.getQualifierLength()) == 0) {
                LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
                if (linkType == LinkType.INDEX_TABLE) {
                    addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes, clientVersion);
                } else if (linkType == PHYSICAL_TABLE) {
                    // famName contains the logical name of the parent table. We need to get the actual physical name of the table
                    PTable parentTable = null;
                    // call getTable() on famName only if it does not start with _IDX_.
                    // Table name starting with _IDX_ always must refer to HBase table that is
                    // shared by all view indexes on the given table/view hierarchy.
                    // _IDX_ is HBase table that does not have corresponding PTable representation
                    // in Phoenix, hence there is no point of calling getTable().
                    if (!famName.getString().startsWith(MetaDataUtil.VIEW_INDEX_TABLE_PREFIX)
                        && indexType != IndexType.LOCAL) {
                        try {
                            parentTable = doGetTable(null,
                                SchemaUtil.getSchemaNameFromFullName(famName.getBytes())
                                    .getBytes(StandardCharsets.UTF_8),
                                SchemaUtil.getTableNameFromFullName(famName.getBytes())
                                    .getBytes(StandardCharsets.UTF_8), clientTimeStamp,
                                clientVersion);
                        } catch (SQLException e) {
                            if (e.getErrorCode()
                                != SQLExceptionCode.GET_TABLE_ERROR.getErrorCode()) {
                                LOGGER.error(
                                    "Error while retrieving getTable for PHYSICAL_TABLE link to {}",
                                    famName, e);
                                throw e;
                            }
                        }
                        if (isSystemCatalogSplittable
                            && (parentTable == null || isTableDeleted(parentTable))) {
                            // parentTable is neither in the cache nor in the local region. Since
                            // famName is only logical name, we need to find the physical table.
                            // Hence, it is recommended to scan SYSTEM.CATALOG table again using
                            // separate CQSI connection as SYSTEM.CATALOG is splittable so the
                            // PTable with famName might be available on different region.
                            try (PhoenixConnection connection = getServerConnectionForMetaData(
                                    env.getConfiguration()).unwrap(PhoenixConnection.class)) {
                                parentTable = connection.getTableNoCache(famName.getString());
                            } catch (TableNotFoundException e) {
                                // It is ok to swallow this exception since this could be a view index and _IDX_ table is not there.
                            }
                        }
                    }

                    if (parentTable == null || isTableDeleted(parentTable)) {
                        if (indexType == IndexType.LOCAL) {
                            PName tablePhysicalName = getPhysicalTableName(
                                env.getRegion(),null,
                                SchemaUtil.getSchemaNameFromFullName(
                                    famName.getBytes()).getBytes(StandardCharsets.UTF_8),
                                SchemaUtil.getTableNameFromFullName(
                                    famName.getBytes()).getBytes(StandardCharsets.UTF_8),
                                clientTimeStamp);
                            if (tablePhysicalName == null) {
                                physicalTables.add(famName);
                                setPhysicalName = true;
                            } else {
                                physicalTables.add(SchemaUtil.getPhysicalHBaseTableName(schemaName, tablePhysicalName, isNamespaceMapped));
                                setPhysicalName = true;
                            }
                        } else {
                            physicalTables.add(famName);
                            setPhysicalName = true;
                        }
                        // If this is a view index, then one of the link is IDX_VW -> _IDX_ PhysicalTable link. Since famName is _IDX_ and we can't get this table hence it is null, we need to use actual view name
                        parentLogicalName = (tableType == INDEX ? SchemaUtil.getTableName(parentSchemaName, parentTableName) : famName);
                    } else {
                        String parentPhysicalTableName = parentTable.getPhysicalName().getString();
                        physicalTables.add(PNameFactory.newName(parentPhysicalTableName));
                        setPhysicalName = true;
                        parentLogicalName = SchemaUtil.getTableName(parentTable.getSchemaName(), parentTable.getTableName());
                    }
                } else if (linkType == PARENT_TABLE) {
                    parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
                    parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
                } else if (linkType == LinkType.EXCLUDED_COLUMN) {
                    // add the excludedColumn
                    addExcludedColumnToTable(columns, colName, famName, colKv.getTimestamp());
                } else if (linkType == LinkType.TRANSFORMING_NEW_TABLE) {
                    transformingNewTable = doGetTable((tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes())
                            , SchemaUtil.getSchemaNameFromFullName(famName.getBytes()).getBytes(), SchemaUtil.getTableNameFromFullName(famName.getBytes()).getBytes(), clientTimeStamp, null, clientVersion);
                    if (transformingNewTable == null) {
                        // It could be global
                        transformingNewTable = doGetTable(ByteUtil.EMPTY_BYTE_ARRAY
                                , SchemaUtil.getSchemaNameFromFullName(famName.getBytes()).getBytes(), SchemaUtil.getTableNameFromFullName(famName.getBytes()).getBytes(), clientTimeStamp, null, clientVersion);
                        if (transformingNewTable == null) {
                            ClientUtil.throwIOException("Transforming new table not found", new TableNotFoundException(schemaName.getString(), famName.getString()));
                        }
                    }
                } else if (linkType == VIEW_INDEX_PARENT_TABLE) {
                    byte[] viewKey = getTableKey(tenantId == null ? null : tenantId.getBytes(),
                            parentSchemaName == null ? null : parentSchemaName.getBytes(),
                            parentTableName.getBytes());
                    //parentViewType should not be Mapped
                    ttl = getTTLFromHierarchy(viewKey, clientTimeStamp, true);
                    isThisAViewIndex = true;
                }
            } else {
                long columnTimestamp =
                    columnCellList.get(0).getTimestamp() != HConstants.LATEST_TIMESTAMP ?
                        columnCellList.get(0).getTimestamp() : timeStamp;
                boolean isSalted = saltBucketNum != null
                    || (oldTable != null &&
                    oldTable.getBucketNum() != null && oldTable.getBucketNum() > 0);
                addColumnToTable(columnCellList, colName, famName, colKeyValues, columns,
                        isSalted, baseColumnCount, isRegularView, columnTimestamp);
            }
        }
        // Ignoring meta indexes when looking for TTL on parent tables.
        // Due to failures in Namespace related ITs when isNamespaceMappingEnabled is enabled.
        boolean isMetaIndex = (QueryConstants.SYSTEM_SCHEMA_NAME.equals(schemaName.getString())
                && (tableType == INDEX));
        if (tableType == INDEX && !isThisAViewIndex && ttl.equals(TTL_EXPRESSION_NOT_DEFINED) && !isMetaIndex) {
            //If this is an index on Table get TTL from Table
            byte[] tableKey = getTableKey(tenantId == null ? null : tenantId.getBytes(),
                    parentSchemaName == null ? null : parentSchemaName.getBytes(),
                    parentTableName.getBytes());
            ttl = getTTLForTable(tableKey, clientTimeStamp);
        }
        if (tableType == INDEX
                && CDCUtil.isCDCIndex(tableName.getString())
                && !ttl.equals(TTL_EXPRESSION_NOT_DEFINED)) {
            ttl = TTL_EXPRESSION_FOREVER;
        }
        builder.setTTL(ttl);
        builder.setEncodedCQCounter(cqCounter);

        builder.setIndexes(indexes != null ? indexes : oldTable != null
            ? oldTable.getIndexes() : Collections.<PTable>emptyList());

        if (physicalTables == null || physicalTables.size() == 0) {
            builder.setPhysicalNames(oldTable != null ? oldTable.getPhysicalNames()
                : ImmutableList.<PName>of());
        } else {
            builder.setPhysicalNames(ImmutableList.copyOf(physicalTables));
        }
        if (!setPhysicalName && oldTable != null) {
            builder.setPhysicalTableName(oldTable.getPhysicalName(true));
        }
        builder.setTransformingNewTable(transformingNewTable);

        builder.setExcludedColumns(ImmutableList.<PColumn>of());
        builder.setBaseTableLogicalName(parentLogicalName != null ?
            parentLogicalName : oldTable != null ? oldTable.getBaseTableLogicalName() : null);
        builder.setParentTableName(parentTableName != null ?
            parentTableName : oldTable != null ? oldTable.getParentTableName() : null);
        builder.setParentSchemaName(parentSchemaName != null ? parentSchemaName :
            oldTable != null ? oldTable.getParentSchemaName() : null);

        builder.addOrSetColumns(columns);
        // Avoid querying the stats table because we're holding the rowLock here. Issuing an RPC to a remote
        // server while holding this lock is a bad idea and likely to cause contention.
        return builder.build();
    }