private Map separateAndValidateProperties()

in phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java [3041:3370]


    private Map<TableDescriptor, TableDescriptor> separateAndValidateProperties(PTable table,
            Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded,
             Map<String, Object> tableProps) throws SQLException {
        Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
        Map<String,Object> commonFamilyProps = new HashMap<>();
        boolean addingColumns = colFamiliesForPColumnsToBeAdded != null && !colFamiliesForPColumnsToBeAdded.isEmpty();
        HashSet<String> existingColumnFamilies = existingColumnFamilies(table);
        Map<String, Map<String, Object>> allFamiliesProps = new HashMap<>(existingColumnFamilies.size());
        boolean isTransactional = table.isTransactional();
        boolean willBeTransactional = false;
        boolean isOrWillBeTransactional = isTransactional;
        Integer newTTL = null;
        TTLExpression newPhoenixTTL = null;
        Integer newReplicationScope = null;
        KeepDeletedCells newKeepDeletedCells = null;
        TransactionFactory.Provider txProvider = null;
        for (String family : properties.keySet()) {
            List<Pair<String, Object>> propsList = properties.get(family);
            if (propsList != null && propsList.size() > 0) {
                Map<String, Object> colFamilyPropsMap = new HashMap<>(propsList.size());
                for (Pair<String, Object> prop : propsList) {
                    String propName = prop.getFirst();
                    Object propValue = prop.getSecond();
                    if ((MetaDataUtil.isHTableProperty(propName) ||  TableProperty.isPhoenixTableProperty(propName)) && addingColumns) {
                        // setting HTable and PhoenixTable properties while adding a column is not allowed.
                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_SET_TABLE_PROPERTY_ADD_COLUMN)
                        .setMessage("Property: " + propName)
                        .setSchemaName(table.getSchemaName().getString())
                        .setTableName(table.getTableName().getString())
                        .build()
                        .buildException();
                    }
                    if (MetaDataUtil.isHTableProperty(propName)) {
                        // Can't have a column family name for a property that's an HTableProperty
                        if (!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY)
                            .setMessage("Column Family: " + family + ", Property: " + propName)
                            .setSchemaName(table.getSchemaName().getString())
                            .setTableName(table.getTableName().getString())
                            .build()
                            .buildException();
                        }
                        tableProps.put(propName, propValue);
                    } else {
                        if (TableProperty.isPhoenixTableProperty(propName)) {
                            TableProperty tableProp = TableProperty.valueOf(propName);
                            tableProp.validate(true, !family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY), table.getType());
                            if (propName.equals(TTL)) {
                                if (table.getType() == PTableType.INDEX) {
                                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_SET_OR_ALTER_PROPERTY_FOR_INDEX)
                                            .setMessage("Property: " + propName).build()
                                            .buildException();
                                }
                                //Handle FOREVER and NONE case
                                propValue = convertForeverAndNoneTTLValue(propValue, isPhoenixTTLEnabled());
                                //If Phoenix level TTL is enabled we are using TTL as phoenix
                                //Table level property.
                                if (!isPhoenixTTLEnabled()) {
                                    // only literal TTL expression
                                    LiteralTTLExpression ttlExpr =
                                            (LiteralTTLExpression) TableProperty.TTL.getValue(propValue);
                                    newTTL = ttlExpr != null ? ttlExpr.getTTLValue() : null;
                                    //Even though TTL is really a HColumnProperty we treat it
                                    //specially. We enforce that all CFs have the same TTL.
                                    commonFamilyProps.put(propName, propValue);
                                } else {
                                    //Setting this here just to check if we need to throw Exception
                                    //for Transaction's SET_TTL Feature.
                                    newPhoenixTTL = (TTLExpression) TableProperty.TTL.getValue(propValue);
                                }
                            } else if (propName.equals(PhoenixDatabaseMetaData.TRANSACTIONAL) && Boolean.TRUE.equals(propValue)) {
                                willBeTransactional = isOrWillBeTransactional = true;
                                tableProps.put(PhoenixTransactionContext.READ_NON_TX_DATA, propValue);
                            } else if (propName.equals(PhoenixDatabaseMetaData.TRANSACTION_PROVIDER) && propValue != null) {
                                willBeTransactional = isOrWillBeTransactional = true;
                                tableProps.put(PhoenixTransactionContext.READ_NON_TX_DATA, Boolean.TRUE);
                                txProvider = (Provider)TableProperty.TRANSACTION_PROVIDER.getValue(propValue);
                                tableProps.put(PhoenixDatabaseMetaData.TRANSACTION_PROVIDER, txProvider);
                            }
                        } else {
                            if (MetaDataUtil.isHColumnProperty(propName)) {
                                if (table.getType() == PTableType.INDEX && MetaDataUtil.propertyNotAllowedToBeOutOfSync(propName)) {
                                    // We disallow index tables from overriding TTL, KEEP_DELETED_CELLS and REPLICATION_SCOPE,
                                    // in order to avoid situations where indexes are not in sync with their data table
                                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_SET_OR_ALTER_PROPERTY_FOR_INDEX)
                                            .setMessage("Property: " + propName).build()
                                            .buildException();
                                }
                                if (family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                                    if (propName.equals(KEEP_DELETED_CELLS)) {
                                        newKeepDeletedCells =
                                                Boolean.valueOf(propValue.toString()) ? KeepDeletedCells.TRUE : KeepDeletedCells.FALSE;
                                    }
                                    if (propName.equals(REPLICATION_SCOPE)) {
                                        newReplicationScope = ((Number)propValue).intValue();
                                    }
                                    commonFamilyProps.put(propName, propValue);
                                } else if (MetaDataUtil.propertyNotAllowedToBeOutOfSync(propName)) {
                                    // Don't allow specifying column families for TTL, KEEP_DELETED_CELLS and REPLICATION_SCOPE.
                                    // These properties can only be applied for all column families of a table and can't be column family specific.
                                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_FOR_PROPERTY)
                                            .setMessage("Property: " + propName).build()
                                            .buildException();
                                } else {
                                    colFamilyPropsMap.put(propName, propValue);
                                }
                            } else {
                                // invalid property - neither of HTableProp, HColumnProp or PhoenixTableProp
                                // FIXME: This isn't getting triggered as currently a property gets evaluated
                                // as HTableProp if its neither HColumnProp or PhoenixTableProp.
                                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_PROPERTY)
                                .setMessage("Column Family: " + family + ", Property: " + propName)
                                .setSchemaName(table.getSchemaName().getString())
                                .setTableName(table.getTableName().getString())
                                .build()
                                .buildException();
                            }
                        }
                    }
                }
                if (isOrWillBeTransactional && (newTTL != null || newPhoenixTTL != null)) {
                    TransactionFactory.Provider isOrWillBeTransactionProvider = txProvider == null ? table.getTransactionProvider() : txProvider;
                    if (isOrWillBeTransactionProvider.getTransactionProvider().isUnsupported(PhoenixTransactionProvider.Feature.SET_TTL)) {
                        throw new SQLExceptionInfo.Builder(PhoenixTransactionProvider.Feature.SET_TTL.getCode())
                        .setMessage(isOrWillBeTransactionProvider.name())
                        .setSchemaName(table.getSchemaName().getString())
                        .setTableName(table.getTableName().getString())
                        .build()
                        .buildException();
                    }
                }
                if (!colFamilyPropsMap.isEmpty()) {
                    stmtFamiliesPropsMap.put(family, colFamilyPropsMap);
                }

            }
        }
        commonFamilyProps = Collections.unmodifiableMap(commonFamilyProps);
        boolean isAddingPkColOnly = colFamiliesForPColumnsToBeAdded.size() == 1 && colFamiliesForPColumnsToBeAdded.contains(null);
        if (!commonFamilyProps.isEmpty()) {
            if (!addingColumns) {
                // Add the common family props to all existing column families
                for (String existingColFamily : existingColumnFamilies) {
                    Map<String, Object> m = new HashMap<>(commonFamilyProps.size());
                    m.putAll(commonFamilyProps);
                    allFamiliesProps.put(existingColFamily, m);
                }
            } else {
                // Add the common family props to the column families of the columns being added
                for (String colFamily : colFamiliesForPColumnsToBeAdded) {
                    if (colFamily != null) {
                        // only set properties for key value columns
                        Map<String, Object> m = new HashMap<>(commonFamilyProps.size());
                        m.putAll(commonFamilyProps);
                        allFamiliesProps.put(colFamily, m);
                    } else if (isAddingPkColOnly) {
                        // Setting HColumnProperty for a pk column is invalid
                        // because it will be part of the row key and not a key value column family.
                        // However, if both pk cols as well as key value columns are getting added
                        // together, then its allowed. The above if block will make sure that we add properties
                        // only for the kv cols and not pk cols.
                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
                        .build().buildException();
                    }
                }
            }
        }

        // Now go through the column family properties specified in the statement
        // and merge them with the common family properties.
        for (String f : stmtFamiliesPropsMap.keySet()) {
            if (!addingColumns && !existingColumnFamilies.contains(f)) {
                String schemaNameStr = table.getSchemaName()==null?null:table.getSchemaName().getString();
                String tableNameStr = table.getTableName()==null?null:table.getTableName().getString();
                throw new ColumnFamilyNotFoundException(schemaNameStr, tableNameStr, f);
            }
            if (addingColumns && !colFamiliesForPColumnsToBeAdded.contains(f)) {
                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_SET_PROPERTY_FOR_COLUMN_NOT_ADDED).build().buildException();
            }
            Map<String, Object> commonProps = allFamiliesProps.get(f);
            Map<String, Object> stmtProps = stmtFamiliesPropsMap.get(f);
            if (commonProps != null) {
                if (stmtProps != null) {
                    // merge common props with statement props for the family
                    commonProps.putAll(stmtProps);
                }
            } else {
                // if no common props were specified, then assign family specific props
                if (stmtProps != null) {
                    allFamiliesProps.put(f, stmtProps);
                }
            }
        }

        // case when there is a column family being added but there are no props
        // For ex - in DROP COLUMN when a new empty CF needs to be added since all
        // the columns of the existing empty CF are getting dropped. Or the case
        // when one is just adding a column for a column family like this:
        // ALTER TABLE ADD CF.COL
        for (String cf : colFamiliesForPColumnsToBeAdded) {
            if (cf != null && allFamiliesProps.get(cf) == null) {
                allFamiliesProps.put(cf, new HashMap<String, Object>());
            }
        }

        if (table.getColumnFamilies().isEmpty() && !addingColumns && !commonFamilyProps.isEmpty()) {
            allFamiliesProps.put(Bytes.toString(table.getDefaultFamilyName() == null ? QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES : table.getDefaultFamilyName().getBytes() ), commonFamilyProps);
        }

        // Views are not allowed to have any of these properties.
        if (table.getType() == PTableType.VIEW && (!stmtFamiliesPropsMap.isEmpty() || !commonFamilyProps.isEmpty() || !tableProps.isEmpty())) {
            throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
            .buildException();
        }

        TableDescriptorBuilder newTableDescriptorBuilder = null;
        TableDescriptor origTableDescriptor = null;
        // Store all old to new table descriptor mappings for the table as well as its global indexes
        Map<TableDescriptor, TableDescriptor> tableAndIndexDescriptorMappings = Collections.emptyMap();
        if (!allFamiliesProps.isEmpty() || !tableProps.isEmpty()) {
            tableAndIndexDescriptorMappings = Maps.newHashMapWithExpectedSize(3 + table.getIndexes().size());
            TableDescriptor existingTableDescriptor = origTableDescriptor = this.getTableDescriptor(table.getPhysicalName().getBytes());
            newTableDescriptorBuilder = TableDescriptorBuilder.newBuilder(existingTableDescriptor);
            if (!tableProps.isEmpty()) {
                // add all the table properties to the new table descriptor
                for (Entry<String, Object> entry : tableProps.entrySet()) {
                    newTableDescriptorBuilder.setValue(entry.getKey(), entry.getValue() != null ? entry.getValue().toString() : null);
                }
            }
            if (addingColumns) {
                // Make sure that TTL, KEEP_DELETED_CELLS and REPLICATION_SCOPE for the new column family to be added stays in sync
                // with the table's existing column families. Note that we use the new values for these properties in case we are
                // altering their values. We also propagate these altered values to existing column families and indexes on the table below
                setSyncedPropsForNewColumnFamilies(allFamiliesProps, table, newTableDescriptorBuilder, newTTL, newKeepDeletedCells, newReplicationScope);
            }
            if (newTTL != null || newKeepDeletedCells != null || newReplicationScope != null) {
                // Set properties to be kept in sync on all table column families of this table, even if they are not referenced here
                setSyncedPropsForUnreferencedColumnFamilies(this.getTableDescriptor(table.getPhysicalName().getBytes()),
                        allFamiliesProps, newTTL, newKeepDeletedCells, newReplicationScope);
            }

            Integer defaultTxMaxVersions = null;
            if (isOrWillBeTransactional) {
                // Calculate default for max versions
                Map<String, Object> emptyFamilyProps = allFamiliesProps.get(SchemaUtil.getEmptyColumnFamilyAsString(table));
                if (emptyFamilyProps != null) {
                    defaultTxMaxVersions = (Integer)emptyFamilyProps.get(MAX_VERSIONS);
                }
                if (defaultTxMaxVersions == null) {
                    if (isTransactional) {
                        defaultTxMaxVersions = newTableDescriptorBuilder.build()
                                .getColumnFamily(SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions();
                    } else {
                        defaultTxMaxVersions =
                                this.getProps().getInt(
                                        QueryServices.MAX_VERSIONS_TRANSACTIONAL_ATTRIB,
                                        QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL);
                    }
                }
                if (willBeTransactional) {
                    // Set VERSIONS for all column families when transitioning to transactional
                    for (PColumnFamily family : table.getColumnFamilies()) {
                        if (!allFamiliesProps.containsKey(family.getName().getString())) {
                            Map<String,Object> familyProps = Maps.newHashMapWithExpectedSize(1);
                            familyProps.put(MAX_VERSIONS, defaultTxMaxVersions);
                            allFamiliesProps.put(family.getName().getString(), familyProps);
                        }
                    }
                }
                // Set transaction context TTL property based on HBase property if we're
                // transitioning to become transactional or setting TTL on
                // an already transactional table.
                int ttl = getTTL(table, newTableDescriptorBuilder.build(), newTTL);
                if (ttl != ColumnFamilyDescriptorBuilder.DEFAULT_TTL) {
                    for (Map.Entry<String, Map<String, Object>> entry : allFamiliesProps.entrySet()) {
                        Map<String, Object> props = entry.getValue();
                        if (props == null) {
                            allFamiliesProps.put(entry.getKey(), new HashMap<>());
                            props = allFamiliesProps.get(entry.getKey());
                        } else {
                            props = new HashMap<>(props);
                        }
                        // Note: After PHOENIX-6627, is PhoenixTransactionContext.PROPERTY_TTL still useful?
                        props.put(PhoenixTransactionContext.PROPERTY_TTL, ttl);
                        // Remove HBase TTL if we're not transitioning an existing table to become transactional
                        // or if the existing transactional table wasn't originally non transactional.
                        if (!willBeTransactional && !Boolean.valueOf(newTableDescriptorBuilder.build().getValue(PhoenixTransactionContext.READ_NON_TX_DATA))) {
                            props.remove(TTL);
                        }
                        entry.setValue(props);
                    }
                }
            }
            for (Entry<String, Map<String, Object>> entry : allFamiliesProps.entrySet()) {
                Map<String,Object> familyProps = entry.getValue();
                if (isOrWillBeTransactional) {
                    if (!familyProps.containsKey(MAX_VERSIONS)) {
                        familyProps.put(MAX_VERSIONS, defaultTxMaxVersions);
                    }
                }
                byte[] cf = Bytes.toBytes(entry.getKey());
                ColumnFamilyDescriptor colDescriptor = newTableDescriptorBuilder.build().getColumnFamily(cf);
                if (colDescriptor == null) {
                    // new column family
                    colDescriptor = generateColumnFamilyDescriptor(new Pair<>(cf, familyProps), table.getType());
                    newTableDescriptorBuilder.setColumnFamily(colDescriptor);
                } else {
                    ColumnFamilyDescriptorBuilder colDescriptorBuilder = ColumnFamilyDescriptorBuilder.newBuilder(colDescriptor);
                    modifyColumnFamilyDescriptor(colDescriptorBuilder, familyProps);
                    colDescriptor = colDescriptorBuilder.build();
                    newTableDescriptorBuilder.removeColumnFamily(cf);
                    newTableDescriptorBuilder.setColumnFamily(colDescriptor);
                }
                if (isOrWillBeTransactional) {
                    checkTransactionalVersionsValue(colDescriptor);
                }
            }
        }
        if (origTableDescriptor != null && newTableDescriptorBuilder != null) {
            // Add the table descriptor mapping for the base table
            tableAndIndexDescriptorMappings.put(origTableDescriptor, newTableDescriptorBuilder.build());
        }

        Map<String, Object> applyPropsToAllIndexColFams = getNewSyncedPropsMap(newTTL, newKeepDeletedCells, newReplicationScope);
        // Copy properties that need to be synced from the default column family of the base table to
        // the column families of each of its indexes (including indexes on this base table's views)
        // and store those table descriptor mappings as well
        setSyncedPropertiesForTableIndexes(table, tableAndIndexDescriptorMappings, applyPropsToAllIndexColFams);
        return tableAndIndexDescriptorMappings;
    }