You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by clohfink <gi...@git.apache.org> on 2018/03/12 20:58:51 UTC

[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

GitHub user clohfink opened a pull request:

    https://github.com/apache/cassandra/pull/205

    Add virtual tables for CASSANDRA-7622

    

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

    $ git pull https://github.com/clohfink/cassandra 7622-trunk

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

    https://github.com/apache/cassandra/pull/205.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 #205
    
----
commit 170e71e2b0b6e26e7092f9a35b0f166676d073f6
Author: Chris Lohfink <cl...@...>
Date:   2018-03-12T20:53:39Z

    Add virtual tables for CASSANDRA-7622

----


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173964748
  
    --- Diff: src/java/org/apache/cassandra/db/virtual/Settings.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.lang.reflect.Array;
    +import java.lang.reflect.Field;
    +import java.lang.reflect.Modifier;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.function.Consumer;
    +
    +import org.apache.cassandra.config.Config;
    +import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.cql3.CQL3Type;
    +import org.apache.cassandra.cql3.ColumnIdentifier;
    +import org.apache.cassandra.cql3.QueryOptions;
    +import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
    +import org.apache.cassandra.db.DecoratedKey;
    +import org.apache.cassandra.db.InMemoryVirtualTable;
    +import org.apache.cassandra.db.rows.Row;
    +import org.apache.cassandra.exceptions.CassandraException;
    +import org.apache.cassandra.exceptions.InvalidRequestException;
    +import org.apache.cassandra.schema.ColumnMetadata;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.service.StorageProxy;
    +import org.apache.cassandra.service.StorageService;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.ImmutableMap;
    +
    +public class Settings extends InMemoryVirtualTable
    +{
    +    private static final Logger logger = LoggerFactory.getLogger(Settings.class);
    +    private static Map<String, Consumer<String>> WRITABLES = ImmutableMap.<String, Consumer<String>>builder()
    +      .put("batch_size_warn_threshold_in_kb", v -> DatabaseDescriptor.setBatchSizeWarnThresholdInKB(Integer.parseInt(v)))
    +      .put("batch_size_fail_threshold_in_kb", v -> DatabaseDescriptor.setBatchSizeFailThresholdInKB(Integer.parseInt(v)))
    +
    +      .put("compaction_throughput_mb_per_sec", v -> StorageService.instance.setCompactionThroughputMbPerSec(Integer.parseInt(v)))
    +      .put("concurrent_compactors", v -> StorageService.instance.setConcurrentCompactors(Integer.parseInt(v)))
    +      .put("concurrent_validations", v -> StorageService.instance.setConcurrentValidators(Integer.parseInt(v)))
    +
    +      .put("tombstone_warn_threshold", v -> DatabaseDescriptor.setTombstoneWarnThreshold(Integer.parseInt(v)))
    +      .put("tombstone_failure_threshold", v -> DatabaseDescriptor.setTombstoneFailureThreshold(Integer.parseInt(v)))
    +
    +      .put("hinted_handoff_enabled", v -> StorageProxy.instance.setHintedHandoffEnabled(Boolean.parseBoolean(v)))
    +      .put("hinted_handoff_throttle_in_kb", v -> StorageService.instance.setHintedHandoffThrottleInKB(Integer.parseInt(v)))
    +
    +      .put("incremental_backups", v -> DatabaseDescriptor.setIncrementalBackupsEnabled(Boolean.parseBoolean(v)))
    +
    +      .put("inter_dc_stream_throughput_outbound_megabits_per_sec", v -> StorageService.instance.setInterDCStreamThroughputMbPerSec(Integer.parseInt(v)))
    +      .put("stream_throughput_outbound_megabits_per_sec", v -> StorageService.instance.setStreamThroughputMbPerSec(Integer.parseInt(v)))
    +
    +      .put("truncate_request_timeout_in_ms", v -> StorageService.instance.setTruncateRpcTimeout(Long.parseLong(v)))
    +      .put("cas_contention_timeout_in_ms", v -> StorageService.instance.setCasContentionTimeout(Long.parseLong(v)))
    +      .put("counter_write_request_timeout_in_ms", v -> StorageService.instance.setCounterWriteRpcTimeout(Long.parseLong(v)))
    +      .put("write_request_timeout_in_ms", v -> StorageService.instance.setWriteRpcTimeout(Long.parseLong(v)))
    +      .put("range_request_timeout_in_ms", v -> StorageService.instance.setRangeRpcTimeout(Long.parseLong(v)))
    +      .put("read_request_timeout_in_ms", v -> StorageService.instance.setReadRpcTimeout(Long.parseLong(v)))
    +      .put("request_timeout_in_ms",v -> StorageService.instance.setRpcTimeout(Long.parseLong(v)))
    +
    +      .put("phi_convict_threshold", v -> DatabaseDescriptor.setPhiConvictThreshold(Double.parseDouble(v)))
    +      .build();
    +
    +    public static Map<String, CQL3Type> columns()
    +    {
    +        Map<String, CQL3Type> definitions = new HashMap<>();
    +        definitions.put("setting", CQL3Type.Native.TEXT);
    +        definitions.put("value", CQL3Type.Native.TEXT);
    +        definitions.put("writable", CQL3Type.Native.BOOLEAN);
    +        return definitions;
    +    }
    +
    +    public static Key primaryKey()
    +    {
    +        return createKey().addKey("setting");
    +    }
    +
    +    ColumnMetadata valueColumn;
    +    public Settings(TableMetadata metadata)
    +    {
    +        super(metadata);
    +        valueColumn = metadata.getColumn(ColumnIdentifier.getInterned("value", false));
    +    }
    +
    +    public boolean writable()
    +    {
    +        return true;
    +    }
    +
    +    /**
    +     * Execute an update operation.
    +     *
    +     * @param partitionKey partition key for the update.
    +     * @param params parameters of the update.
    +     */
    +    @Override
    +    public void mutate(DecoratedKey partitionKey, Row row) throws CassandraException
    +    {
    +        String setting = metadata.partitionKeyType.getString(partitionKey.getKey());
    +        if (WRITABLES.get(setting) == null)
    +            throw new InvalidRequestException(setting + " is not a writable setting.");
    +        if (row.getCell(valueColumn) == null)
    +            throw new InvalidRequestException("Only 'value' is updatable.");
    +
    +        String value = valueColumn.type.getString(row.getCell(valueColumn).value());
    +        WRITABLES.get(setting).accept(value);
    +    }
    +
    +    public void read(StatementRestrictions restrictions, QueryOptions options, ResultBuilder result)
    +    {
    +        Field[] fields = Config.class.getFields();
    +        Config config = DatabaseDescriptor.getRawConfig();
    +        for (Field f : fields)
    +        {
    +            if (!Modifier.isStatic(f.getModifiers()))
    +            {
    +                try
    +                {
    +                    Object value = f.get(config);
    +                    if (value != null && value.getClass().isArray())
    +                    {
    +                        String s = "[";
    --- End diff --
    
    Dude, `StringBuilder`!!


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173963909
  
    --- Diff: src/java/org/apache/cassandra/db/virtual/CompactionStats.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.HashMap;
    +import java.util.Map;
    +import java.util.UUID;
    +
    +import org.apache.cassandra.cql3.CQL3Type;
    +import org.apache.cassandra.cql3.QueryOptions;
    +import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
    +import org.apache.cassandra.db.InMemoryVirtualTable;
    +import org.apache.cassandra.db.compaction.CompactionManager;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.service.StorageService;
    +
    +public class CompactionStats extends InMemoryVirtualTable
    +{
    +
    +    public static Map<String, CQL3Type> columns()
    --- End diff --
    
    change this to a static initializer if it's never going to change. 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173966458
  
    --- Diff: src/java/org/apache/cassandra/schema/TableMetadata.java ---
    @@ -552,22 +601,25 @@ public boolean equals(Object o)
             TableMetadata tm = (TableMetadata) o;
     
             return keyspace.equals(tm.keyspace)
    -            && name.equals(tm.name)
    -            && id.equals(tm.id)
    -            && partitioner.equals(tm.partitioner)
    -            && params.equals(tm.params)
    -            && flags.equals(tm.flags)
    -            && isView == tm.isView
    -            && columns.equals(tm.columns)
    -            && droppedColumns.equals(tm.droppedColumns)
    -            && indexes.equals(tm.indexes)
    -            && triggers.equals(tm.triggers);
    +                && name.equals(tm.name)
    +                && id.equals(tm.id)
    +                && partitioner.equals(tm.partitioner)
    +                && params.equals(tm.params)
    +                && flags.equals(tm.flags)
    +                && isView == tm.isView
    +                && columns.equals(tm.columns)
    +                && droppedColumns.equals(tm.droppedColumns)
    +                && indexes.equals(tm.indexes)
    +                && triggers.equals(tm.triggers)
    +                && isVirtual == tm.isVirtual
    +                && (virtualKlass == null || virtualKlass.equals(tm.virtualKlass));
    --- End diff --
    
    null class a legal state?


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173957234
  
    --- Diff: src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java ---
    @@ -195,11 +217,27 @@ public TableMetadata toTableMetadata()
             private final Multiset<ColumnIdentifier> definedNames = HashMultiset.create(1);
     
             private final boolean ifNotExists;
    +        private final boolean isVirtual;
    +        private final String klass;
     
             public RawStatement(CFName name, boolean ifNotExists)
             {
                 super(name);
                 this.ifNotExists = ifNotExists;
    +            this.isVirtual = false;
    +            this.klass = null;
    +
    +            assert isVirtual == (klass != null);
    --- End diff --
    
    Sorta OT, but we should just stop doing this and use `Preconditions` to throw IAEs. I hate the fact that a user can disable so much of our error checking w. a command line arg. 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173965027
  
    --- Diff: src/java/org/apache/cassandra/db/virtual/TableStats.java ---
    @@ -0,0 +1,166 @@
    +package org.apache.cassandra.db.virtual;
    +
    +import java.lang.reflect.Field;
    +import java.lang.reflect.Modifier;
    +import java.nio.ByteBuffer;
    +import java.util.*;
    +
    +import org.apache.cassandra.cql3.CQL3Type;
    +import org.apache.cassandra.cql3.QueryOptions;
    +import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
    +import org.apache.cassandra.db.ColumnFamilyStore;
    +import org.apache.cassandra.db.InMemoryVirtualTable;
    +import org.apache.cassandra.db.marshal.CompositeType;
    +import org.apache.cassandra.metrics.LatencyMetrics;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.utils.EstimatedHistogram;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.codahale.metrics.Counting;
    +import com.codahale.metrics.Gauge;
    +import com.codahale.metrics.Metric;
    +import com.codahale.metrics.Sampling;
    +import com.codahale.metrics.Snapshot;
    +import com.codahale.metrics.Timer;
    +import com.google.common.collect.ImmutableSet;
    +
    +public class TableStats extends InMemoryVirtualTable
    +{
    +    private static final Logger logger = LoggerFactory.getLogger(TableStats.class);
    +    private static final Field[] FIELDS = org.apache.cassandra.metrics.TableMetrics.class.getFields();
    +    static
    +    {
    +        Arrays.sort(FIELDS, Comparator.comparing(Field::getName));
    +    }
    +    private static final Collection<String> EH_GAUGES = ImmutableSet.of(
    +            "estimatedPartitionSizeHistogram",
    +            "estimatedColumnCountHistogram");
    +    private CompositeType keyType;
    +
    +    public static Map<String, CQL3Type> columns()
    --- End diff --
    
    same as above. Initialize statically. 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173958314
  
    --- Diff: src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java ---
    @@ -267,12 +267,18 @@ public boolean updatesRegularRows()
             // columns is if we set some static columns, and in that case no clustering
             // columns should be given. So in practice, it's enough to check if we have
             // either the table has no clustering or if it has at least one of them set.
    -        return metadata().clusteringColumns().isEmpty() || restrictions.hasClusteringColumnsRestrictions();
    +        return !metadata().isVirtual() &&
    --- End diff --
    
    nit: why use the method if we have it locally (and it's public anyhoo). The object reference is being used elsewhere herein already. 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173964217
  
    --- Diff: src/java/org/apache/cassandra/db/virtual/CompactionStats.java ---
    @@ -0,0 +1,75 @@
    +/*
    + * 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.HashMap;
    +import java.util.Map;
    +import java.util.UUID;
    +
    +import org.apache.cassandra.cql3.CQL3Type;
    +import org.apache.cassandra.cql3.QueryOptions;
    +import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
    +import org.apache.cassandra.db.InMemoryVirtualTable;
    +import org.apache.cassandra.db.compaction.CompactionManager;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.service.StorageService;
    +
    +public class CompactionStats extends InMemoryVirtualTable
    +{
    +
    +    public static Map<String, CQL3Type> columns()
    +    {
    +        Map<String, CQL3Type> definitions = new HashMap<>();
    +        definitions.put("host_id", CQL3Type.Native.UUID);
    +        definitions.put("compaction_id", CQL3Type.Native.UUID);
    +        definitions.put("task_type", CQL3Type.Native.TEXT);
    +        definitions.put("keyspace_name", CQL3Type.Native.TEXT);
    +        definitions.put("table_name", CQL3Type.Native.TEXT);
    +        definitions.put("bytes_compacted", CQL3Type.Native.TEXT);
    +        definitions.put("bytes_total", CQL3Type.Native.TEXT);
    +        return definitions;
    +    }
    +
    +    public static Key primaryKey()
    +    {
    +        return createKey()
    +                .addKey("host_id")
    +                .addClustering("compaction_id");
    +    }
    +
    +    public CompactionStats(TableMetadata metadata)
    +    {
    +        super(metadata);
    +    }
    +
    +    public void read(StatementRestrictions restrictions, QueryOptions options, ResultBuilder result)
    +    {
    +        UUID hostId = StorageService.instance.getLocalHostUUID();
    +        for (Map<String, String> c : CompactionManager.instance.getCompactions())
    +        {
    +            result.row(hostId, UUID.fromString(c.get("compactionId")))
    --- End diff --
    
    My $0.02 is that if we are duping the same strings everywhere, we should do some statics we don't have a whole nudist colony of naked literals. 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

Posted by clohfink <gi...@git.apache.org>.
Github user clohfink closed the pull request at:

    https://github.com/apache/cassandra/pull/205


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173964414
  
    --- Diff: src/java/org/apache/cassandra/db/virtual/Settings.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.lang.reflect.Array;
    +import java.lang.reflect.Field;
    +import java.lang.reflect.Modifier;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.function.Consumer;
    +
    +import org.apache.cassandra.config.Config;
    +import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.cql3.CQL3Type;
    +import org.apache.cassandra.cql3.ColumnIdentifier;
    +import org.apache.cassandra.cql3.QueryOptions;
    +import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
    +import org.apache.cassandra.db.DecoratedKey;
    +import org.apache.cassandra.db.InMemoryVirtualTable;
    +import org.apache.cassandra.db.rows.Row;
    +import org.apache.cassandra.exceptions.CassandraException;
    +import org.apache.cassandra.exceptions.InvalidRequestException;
    +import org.apache.cassandra.schema.ColumnMetadata;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.service.StorageProxy;
    +import org.apache.cassandra.service.StorageService;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.ImmutableMap;
    +
    +public class Settings extends InMemoryVirtualTable
    +{
    +    private static final Logger logger = LoggerFactory.getLogger(Settings.class);
    +    private static Map<String, Consumer<String>> WRITABLES = ImmutableMap.<String, Consumer<String>>builder()
    +      .put("batch_size_warn_threshold_in_kb", v -> DatabaseDescriptor.setBatchSizeWarnThresholdInKB(Integer.parseInt(v)))
    +      .put("batch_size_fail_threshold_in_kb", v -> DatabaseDescriptor.setBatchSizeFailThresholdInKB(Integer.parseInt(v)))
    +
    +      .put("compaction_throughput_mb_per_sec", v -> StorageService.instance.setCompactionThroughputMbPerSec(Integer.parseInt(v)))
    +      .put("concurrent_compactors", v -> StorageService.instance.setConcurrentCompactors(Integer.parseInt(v)))
    +      .put("concurrent_validations", v -> StorageService.instance.setConcurrentValidators(Integer.parseInt(v)))
    +
    +      .put("tombstone_warn_threshold", v -> DatabaseDescriptor.setTombstoneWarnThreshold(Integer.parseInt(v)))
    +      .put("tombstone_failure_threshold", v -> DatabaseDescriptor.setTombstoneFailureThreshold(Integer.parseInt(v)))
    +
    +      .put("hinted_handoff_enabled", v -> StorageProxy.instance.setHintedHandoffEnabled(Boolean.parseBoolean(v)))
    +      .put("hinted_handoff_throttle_in_kb", v -> StorageService.instance.setHintedHandoffThrottleInKB(Integer.parseInt(v)))
    +
    +      .put("incremental_backups", v -> DatabaseDescriptor.setIncrementalBackupsEnabled(Boolean.parseBoolean(v)))
    +
    +      .put("inter_dc_stream_throughput_outbound_megabits_per_sec", v -> StorageService.instance.setInterDCStreamThroughputMbPerSec(Integer.parseInt(v)))
    +      .put("stream_throughput_outbound_megabits_per_sec", v -> StorageService.instance.setStreamThroughputMbPerSec(Integer.parseInt(v)))
    +
    +      .put("truncate_request_timeout_in_ms", v -> StorageService.instance.setTruncateRpcTimeout(Long.parseLong(v)))
    +      .put("cas_contention_timeout_in_ms", v -> StorageService.instance.setCasContentionTimeout(Long.parseLong(v)))
    +      .put("counter_write_request_timeout_in_ms", v -> StorageService.instance.setCounterWriteRpcTimeout(Long.parseLong(v)))
    +      .put("write_request_timeout_in_ms", v -> StorageService.instance.setWriteRpcTimeout(Long.parseLong(v)))
    +      .put("range_request_timeout_in_ms", v -> StorageService.instance.setRangeRpcTimeout(Long.parseLong(v)))
    +      .put("read_request_timeout_in_ms", v -> StorageService.instance.setReadRpcTimeout(Long.parseLong(v)))
    +      .put("request_timeout_in_ms",v -> StorageService.instance.setRpcTimeout(Long.parseLong(v)))
    +
    +      .put("phi_convict_threshold", v -> DatabaseDescriptor.setPhiConvictThreshold(Double.parseDouble(v)))
    +      .build();
    +
    +    public static Map<String, CQL3Type> columns()
    --- End diff --
    
    same as above - let's just keep a static ref in an initializer. 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173963715
  
    --- Diff: src/java/org/apache/cassandra/db/marshal/AbstractType.java ---
    @@ -529,4 +529,9 @@ public void checkComparable()
         {
             return testAssignment(receiver.type);
         }
    +
    +    public ByteBuffer unsafeDecompose(Object object)
    --- End diff --
    
    I dunno... that a pretty shoot-self-in-foot thing to add the the marshalling API. What's the reasoning?


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173966104
  
    --- Diff: src/java/org/apache/cassandra/schema/TableMetadata.java ---
    @@ -141,6 +147,15 @@ private TableMetadata(Builder builder)
             params = builder.params.build();
             isView = builder.isView;
     
    +        isVirtual = builder.isVirtual;
    +        if (isVirtual)
    +            virtualKlass = builder.virtualKlass;
    +        else
    +            virtualKlass = null;
    +
    +        assert isVirtual == flags.contains(Flag.VIRTUAL) || !Strings.isNullOrEmpty(virtualKlass) && !flags.contains(Flag.VIRTUAL)
    --- End diff --
    
    Same as above w. the asserts. Let's just draw a line in the sand and stop doing this. 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173964951
  
    --- Diff: src/java/org/apache/cassandra/db/virtual/SystemInfoKeyspace.java ---
    @@ -0,0 +1,63 @@
    +package org.apache.cassandra.db.virtual;
    +
    +import static java.lang.String.format;
    +
    +import java.util.List;
    +
    +import org.apache.cassandra.cql3.statements.CreateTableStatement;
    +import org.apache.cassandra.schema.KeyspaceMetadata;
    +import org.apache.cassandra.schema.KeyspaceParams;
    +import org.apache.cassandra.schema.SchemaConstants;
    +import org.apache.cassandra.schema.TableId;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.schema.Tables;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.ImmutableList;
    +
    +public class SystemInfoKeyspace
    +{
    +    private static final Logger logger = LoggerFactory.getLogger(SystemInfoKeyspace.class);
    +
    +    private SystemInfoKeyspace() {}
    +
    +    public static final String SETTINGS = "settings";
    --- End diff --
    
    yay! statics!


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173962979
  
    --- Diff: src/java/org/apache/cassandra/db/VirtualTable.java ---
    @@ -0,0 +1,148 @@
    +/*
    + * 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;
    +
    +import static java.lang.String.format;
    +
    +import java.lang.reflect.InvocationTargetException;
    +import java.util.List;
    +
    +import org.apache.cassandra.cql3.Operation;
    +import org.apache.cassandra.cql3.QueryOptions;
    +import org.apache.cassandra.cql3.UpdateParameters;
    +import org.apache.cassandra.cql3.statements.SelectStatement;
    +import org.apache.cassandra.db.filter.DataLimits;
    +import org.apache.cassandra.db.partitions.PartitionUpdate;
    +import org.apache.cassandra.db.rows.Row;
    +import org.apache.cassandra.exceptions.CassandraException;
    +import org.apache.cassandra.exceptions.ConfigurationException;
    +import org.apache.cassandra.exceptions.InvalidRequestException;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.transport.messages.ResultMessage;
    +import org.apache.cassandra.utils.FBUtilities;
    +
    +import com.google.common.collect.Lists;
    +
    +/**
    + * Base requirements for a VirtualTable. This is required to provide metadata about the virtual table, such as the
    + * partition and clustering keys, and provide a ReadQuery for a SelectStatement.
    + */
    +public abstract class VirtualTable
    +{
    +    protected final TableMetadata metadata;
    +    protected String keyspace;
    +    protected String name;
    +
    +    public VirtualTable(TableMetadata metadata)
    +    {
    +        this.metadata = metadata;
    +    }
    +
    +    public String getTableName()
    +    {
    +        return this.metadata.name;
    +    }
    +
    +    /**
    +     * Is this table writable?
    +     *
    +     * @return True if UPDATE is supported
    +     */
    +    public boolean writable()
    +    {
    +        return false;
    +    }
    +
    +    /**
    +     * If the table allows unrestricted queries (ie filter on clustering key with no partition). Since These tables are
    +     * not backed by the C* data model, this restriction isnt always necessary.
    +     */
    +    public boolean allowFiltering()
    +    {
    +        return true;
    +    }
    +
    +    /**
    +     * Return some implementation of a ReadQuery for a given select statement and query options.
    +     * 
    +     * @param selectStatement
    +     * @param options
    +     * @param limits
    +     * @param nowInSec
    +     * @return ReadQuery
    +     */
    +    public abstract ReadQuery getQuery(SelectStatement selectStatement, QueryOptions options, DataLimits limits,
    +            int nowInSec);
    +
    +    /**
    +     * Execute an update operation.
    +     *
    +     * @param partitionKey
    +     *            partition key for the update.
    +     * @param params
    +     *            parameters of the update.
    +     */
    +    public void mutate(DecoratedKey partitionKey, Row row) throws CassandraException
    +    {
    +        // this should not be called unless writable is overridden
    +        throw new InvalidRequestException("Not Implemented");
    +    }
    +
    +    public static Class<? extends VirtualTable> classFromName(String name)
    +    {
    +        String className = name.contains(".")
    +                ? name
    +                : "org.apache.cassandra.db.virtual." + name;
    +        Class<VirtualTable> strategyClass = FBUtilities.classForName(className, "virtual table");
    +
    +        if (!VirtualTable.class.isAssignableFrom(strategyClass))
    +        {
    +            throw new ConfigurationException(format("Compaction strategy class %s is not derived from VirtualTable",
    +                    className));
    +        }
    +
    +        return strategyClass;
    +    }
    +
    +    public static Key createKey()
    +    {
    +        return new Key();
    --- End diff --
    
    Does it make sense to allow empty key creation as API? If not, switch it to a builder so's it can be validated. 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173965848
  
    --- Diff: src/java/org/apache/cassandra/schema/Schema.java ---
    @@ -713,20 +727,24 @@ private void dropView(ViewMetadata metadata)
     
         private void dropTable(TableMetadata metadata)
         {
    -        ColumnFamilyStore cfs = Keyspace.open(metadata.keyspace).getColumnFamilyStore(metadata.name);
    -        assert cfs != null;
    -        // make sure all the indexes are dropped, or else.
    -        cfs.indexManager.markAllIndexesRemoved();
    -        CompactionManager.instance.interruptCompactionFor(Collections.singleton(metadata), true);
    -        if (DatabaseDescriptor.isAutoSnapshot())
    -            cfs.snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(cfs.name, ColumnFamilyStore.SNAPSHOT_DROP_PREFIX));
    -        CommitLog.instance.forceRecycleAllSegments(Collections.singleton(metadata.id));
    --- End diff --
    
    I'll go back and look through comments, but _can_ i drop a VT? 


---

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


[GitHub] cassandra pull request #205: Add virtual tables for CASSANDRA-7622

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

    https://github.com/apache/cassandra/pull/205#discussion_r173962324
  
    --- Diff: src/java/org/apache/cassandra/db/Keyspace.java ---
    @@ -331,8 +337,11 @@ private Keyspace(String keyspaceName, boolean loadSSTables)
             this.viewManager = new ViewManager(this);
             for (TableMetadata cfm : metadata.tablesAndViews())
             {
    -            logger.trace("Initializing {}.{}", getName(), cfm.name);
    -            initCf(Schema.instance.getTableMetadataRef(cfm.id), loadSSTables);
    +            logger.info("Initializing {}.{}", getName(), cfm.name);
    --- End diff --
    
    Change from trace() to info() intentional?


---

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