You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "David Capwell (Jira)" <ji...@apache.org> on 2022/02/09 17:16:00 UTC

[jira] [Comment Edited] (CASSANDRA-15254) Allow UPDATE on settings virtual table to change running configurations

    [ https://issues.apache.org/jira/browse/CASSANDRA-15254?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17489689#comment-17489689 ] 

David Capwell edited comment on CASSANDRA-15254 at 2/9/22, 5:15 PM:
--------------------------------------------------------------------

bq. so it might not be before a couple of weeks

No worries, wanted to have you look as we talked about this before, will have others review, but wanted to share the SettingsTable class

{code}
final class SettingsTable extends AbstractVirtualTable
{
    private static final String NAME = "name";
    private static final String VALUE = "value";

    protected static final Map<String, Property> PROPERTIES = ImmutableMap.copyOf(getProperties());

    private final Config config;

    SettingsTable(String keyspace)
    {
        this(keyspace, DatabaseDescriptor.getRawConfig());
    }

    SettingsTable(String keyspace, Config config)
    {
        super(TableMetadata.builder(keyspace, "settings")
                           .comment("current settings")
                           .kind(TableMetadata.Kind.VIRTUAL)
                           .partitioner(new LocalPartitioner(UTF8Type.instance))
                           .addPartitionKeyColumn(NAME, UTF8Type.instance)
                           .addRegularColumn(VALUE, UTF8Type.instance)
                           .build());
        this.config = config;
    }

    @Override
    public DataSet data(DecoratedKey partitionKey)
    {
        SimpleDataSet result = new SimpleDataSet(metadata());
        String name = UTF8Type.instance.compose(partitionKey.getKey());
        if (PROPERTIES.containsKey(name))
            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
        return result;
    }

    @Override
    public DataSet data()
    {
        SimpleDataSet result = new SimpleDataSet(metadata());
        for (Map.Entry<String, Property> e : PROPERTIES.entrySet())
            result.row(e.getKey()).column(VALUE, getValue(e.getValue()));
        return result;
    }

    private String getValue(Property prop)
    {
        Object value = prop.get(config);
        return value == null ? null : value.toString();
    }

    private static Map<String, Property> getProperties()
    {
        //TODO possible to move away from "_" to "."?  yaml supports "." so this can cause an inconsistency
        // also, with "_" it isn't clear that a value is in fact nested, so could cause confussion...
        Map<String, Property> properties = Properties.defaultLoader().flatten(Config.class, "_");
        Map<String, Replacement> replacements = Replacements.getNameReplacements(Config.class)
                                                            .get(Config.class);
        if (replacements != null)
        {
            for (Replacement r : replacements.values())
            {
                Property latest = properties.get(r.newName);
                assert latest != null: String.format("Unable to find replacement new name: " + r.newName);
                Property conflict = properties.put(r.oldName, r.toProperty(latest));
                // some configs kept the same name, but changed the type, if this is detected then rely on the replaced property
                assert conflict == null || r.oldName.equals(r.newName): String.format("Multiple properties exist for " + r.oldName);
            }
        }
        return properties;
    }
}
{code}

This is not backwards compatible as there were a few special cased configs that used a different name than the yaml (documented in the settings table test).  This also works with every nested property and exposes all.

Property is get/set for each config, so now have a consistent way for YAML and SettingTable to interact the same way; the only complication for this ticket is that we work with Strings and Property works with the actual type, so need to add a conversion before calling set.


was (Author: dcapwell):
No worries, wanted to have you look as we talked about this before.  In the patch the settings table is now

{code}
/*
 * 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.cassandra.db.virtual;

import java.util.Map;

import com.google.common.collect.ImmutableMap;

import org.apache.cassandra.config.Config;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Properties;
import org.apache.cassandra.config.Replacement;
import org.apache.cassandra.config.Replacements;
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.dht.LocalPartitioner;
import org.apache.cassandra.schema.TableMetadata;
import org.yaml.snakeyaml.introspector.Property;

final class SettingsTable extends AbstractVirtualTable
{
    private static final String NAME = "name";
    private static final String VALUE = "value";

    protected static final Map<String, Property> PROPERTIES = ImmutableMap.copyOf(getProperties());

    private final Config config;

    SettingsTable(String keyspace)
    {
        this(keyspace, DatabaseDescriptor.getRawConfig());
    }

    SettingsTable(String keyspace, Config config)
    {
        super(TableMetadata.builder(keyspace, "settings")
                           .comment("current settings")
                           .kind(TableMetadata.Kind.VIRTUAL)
                           .partitioner(new LocalPartitioner(UTF8Type.instance))
                           .addPartitionKeyColumn(NAME, UTF8Type.instance)
                           .addRegularColumn(VALUE, UTF8Type.instance)
                           .build());
        this.config = config;
    }

    @Override
    public DataSet data(DecoratedKey partitionKey)
    {
        SimpleDataSet result = new SimpleDataSet(metadata());
        String name = UTF8Type.instance.compose(partitionKey.getKey());
        if (PROPERTIES.containsKey(name))
            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
        return result;
    }

    @Override
    public DataSet data()
    {
        SimpleDataSet result = new SimpleDataSet(metadata());
        for (Map.Entry<String, Property> e : PROPERTIES.entrySet())
            result.row(e.getKey()).column(VALUE, getValue(e.getValue()));
        return result;
    }

    private String getValue(Property prop)
    {
        Object value = prop.get(config);
        return value == null ? null : value.toString();
    }

    private static Map<String, Property> getProperties()
    {
        //TODO possible to move away from "_" to "."?  yaml supports "." so this can cause an inconsistency
        // also, with "_" it isn't clear that a value is in fact nested, so could cause confussion...
        Map<String, Property> properties = Properties.defaultLoader().flatten(Config.class, "_");
        Map<String, Replacement> replacements = Replacements.getNameReplacements(Config.class)
                                                            .get(Config.class);
        if (replacements != null)
        {
            for (Replacement r : replacements.values())
            {
                Property latest = properties.get(r.newName);
                assert latest != null: String.format("Unable to find replacement new name: " + r.newName);
                Property conflict = properties.put(r.oldName, r.toProperty(latest));
                // some configs kept the same name, but changed the type, if this is detected then rely on the replaced property
                assert conflict == null || r.oldName.equals(r.newName): String.format("Multiple properties exist for " + r.oldName);
            }
        }
        return properties;
    }
}
{code}

This is not backwards compatible as there were a few special cased configs that used a different name than the yaml (documented in the settings table test).  This also works with every nested property and exposes all.

Property is get/set for each config, so now have a consistent way for YAML and SettingTable to interact the same way; the only complication for this ticket is that we work with Strings and Property works with the actual type, so need to add a conversion before calling set.

> Allow UPDATE on settings virtual table to change running configurations
> -----------------------------------------------------------------------
>
>                 Key: CASSANDRA-15254
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-15254
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Feature/Virtual Tables
>            Reporter: Chris Lohfink
>            Assignee: Benjamin Lerer
>            Priority: Normal
>
> Allow using UPDATE on the system_views.settings virtual table to update configs at runtime for the equivalent of the dispersed JMX attributes/operations.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org