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/10/16 05:40:07 UTC

[GitHub] cassandra pull request #284: Expose schema in virtual table for CASSANDRA-14...

GitHub user clohfink opened a pull request:

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

    Expose schema in virtual table for CASSANDRA-14825

    

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

    $ git pull https://github.com/clohfink/cassandra describe_vtables

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

    https://github.com/apache/cassandra/pull/284.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 #284
    
----
commit 0eccebdeeacaea775ce360b1f183b292f54c9160
Author: Chris Lohfink <cl...@...>
Date:   2018-10-16T05:34:59Z

    Expose schema in virtual table for CASSANDRA-14825

----


---

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


[GitHub] cassandra pull request #284: Expose schema in virtual table for CASSANDRA-14...

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

    https://github.com/apache/cassandra/pull/284#discussion_r232303844
  
    --- Diff: src/java/org/apache/cassandra/db/virtual/DescribeTables.java ---
    @@ -0,0 +1,306 @@
    +/*
    + * 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.nio.ByteBuffer;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Streams;
    +
    +import org.apache.cassandra.cql3.functions.Function;
    +import org.apache.cassandra.cql3.functions.FunctionName;
    +import org.apache.cassandra.cql3.functions.UDAggregate;
    +import org.apache.cassandra.cql3.functions.UDFunction;
    +import org.apache.cassandra.db.DecoratedKey;
    +import org.apache.cassandra.db.Keyspace;
    +import org.apache.cassandra.db.SchemaCQLHelper;
    +import org.apache.cassandra.db.marshal.CompositeType;
    +import org.apache.cassandra.db.marshal.UTF8Type;
    +import org.apache.cassandra.db.marshal.UserType;
    +import org.apache.cassandra.dht.LocalPartitioner;
    +import org.apache.cassandra.schema.Functions;
    +import org.apache.cassandra.schema.IndexMetadata;
    +import org.apache.cassandra.schema.Schema;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.schema.Types;
    +import org.apache.cassandra.schema.ViewMetadata;
    +import org.apache.cassandra.utils.ByteBufferUtil;
    +
    +public class DescribeTables
    +{
    +    private static final String KEYSPACE = "keyspace_name";
    +    private static final String CQL = "cql";
    +
    +    private static final CompositeType utfComposite = CompositeType.getInstance(UTF8Type.instance, UTF8Type.instance);
    +
    +    public static Collection<VirtualTable> getAll(String name)
    +    {
    +        return ImmutableList.of(new DescribeKeyspaceTable(name),
    +                                new DescribeIndexesTable(name),
    +                                new DescribeTypesTable(name),
    +                                new DescribeAggregatesTable(name),
    +                                new DescribeFunctionsTable(name),
    +                                new DescribeViewsTable(name),
    +                                new DescribeTablesTable(name));
    +    }
    +
    +    static final class DescribeKeyspaceTable extends AbstractVirtualTable
    +    {
    +        DescribeKeyspaceTable(String keyspace)
    +        {
    +            super(TableMetadata.builder(keyspace, "describe_keyspace")
    +                               .comment("cql for keyspace metadata")
    +                               .kind(TableMetadata.Kind.VIRTUAL)
    +                               .partitioner(new LocalPartitioner(UTF8Type.instance))
    +                               .addPartitionKeyColumn(KEYSPACE, UTF8Type.instance)
    +                               .addRegularColumn(CQL, UTF8Type.instance)
    +                               .build());
    +        }
    +
    +        @Override
    +        public DataSet data(DecoratedKey partitionKey)
    +        {
    +            String keyspace = UTF8Type.instance.compose(partitionKey.getKey());
    +
    +            SimpleDataSet result = new SimpleDataSet(metadata());
    +            result.row(keyspace)
    +                  .column(CQL, SchemaCQLHelper.getKeyspaceAsCQL(keyspace));
    +            return result;
    +        }
    +
    +        public DataSet data()
    +        {
    +            SimpleDataSet result = new SimpleDataSet(metadata());
    +            for (String keyspace : Schema.instance.getKeyspaces())
    +            {
    +                result.row(keyspace)
    +                      .column(CQL, SchemaCQLHelper.getKeyspaceAsCQL(keyspace));
    +            }
    +            return result;
    +        }
    +    }
    +
    +    static abstract class AbstractDescribeTable extends AbstractVirtualTable
    +    {
    +        AbstractDescribeTable(String keyspace, String name)
    +        {
    +            super(TableMetadata.builder(keyspace, "describe_" + name)
    --- End diff --
    
    Given all these tables will have the same schema, I think it might be more easily used if the “type” was just a column, rather than having to go to a whole new table. So you could just “select * from virtual_schema” and get everything.


---

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


[GitHub] cassandra pull request #284: Expose schema in virtual table for CASSANDRA-14...

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

    https://github.com/apache/cassandra/pull/284#discussion_r232325693
  
    --- Diff: src/java/org/apache/cassandra/db/virtual/DescribeTables.java ---
    @@ -0,0 +1,306 @@
    +/*
    + * 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.nio.ByteBuffer;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Streams;
    +
    +import org.apache.cassandra.cql3.functions.Function;
    +import org.apache.cassandra.cql3.functions.FunctionName;
    +import org.apache.cassandra.cql3.functions.UDAggregate;
    +import org.apache.cassandra.cql3.functions.UDFunction;
    +import org.apache.cassandra.db.DecoratedKey;
    +import org.apache.cassandra.db.Keyspace;
    +import org.apache.cassandra.db.SchemaCQLHelper;
    +import org.apache.cassandra.db.marshal.CompositeType;
    +import org.apache.cassandra.db.marshal.UTF8Type;
    +import org.apache.cassandra.db.marshal.UserType;
    +import org.apache.cassandra.dht.LocalPartitioner;
    +import org.apache.cassandra.schema.Functions;
    +import org.apache.cassandra.schema.IndexMetadata;
    +import org.apache.cassandra.schema.Schema;
    +import org.apache.cassandra.schema.TableMetadata;
    +import org.apache.cassandra.schema.Types;
    +import org.apache.cassandra.schema.ViewMetadata;
    +import org.apache.cassandra.utils.ByteBufferUtil;
    +
    +public class DescribeTables
    +{
    +    private static final String KEYSPACE = "keyspace_name";
    +    private static final String CQL = "cql";
    +
    +    private static final CompositeType utfComposite = CompositeType.getInstance(UTF8Type.instance, UTF8Type.instance);
    +
    +    public static Collection<VirtualTable> getAll(String name)
    +    {
    +        return ImmutableList.of(new DescribeKeyspaceTable(name),
    +                                new DescribeIndexesTable(name),
    +                                new DescribeTypesTable(name),
    +                                new DescribeAggregatesTable(name),
    +                                new DescribeFunctionsTable(name),
    +                                new DescribeViewsTable(name),
    +                                new DescribeTablesTable(name));
    +    }
    +
    +    static final class DescribeKeyspaceTable extends AbstractVirtualTable
    +    {
    +        DescribeKeyspaceTable(String keyspace)
    +        {
    +            super(TableMetadata.builder(keyspace, "describe_keyspace")
    +                               .comment("cql for keyspace metadata")
    +                               .kind(TableMetadata.Kind.VIRTUAL)
    +                               .partitioner(new LocalPartitioner(UTF8Type.instance))
    +                               .addPartitionKeyColumn(KEYSPACE, UTF8Type.instance)
    +                               .addRegularColumn(CQL, UTF8Type.instance)
    +                               .build());
    +        }
    +
    +        @Override
    +        public DataSet data(DecoratedKey partitionKey)
    +        {
    +            String keyspace = UTF8Type.instance.compose(partitionKey.getKey());
    +
    +            SimpleDataSet result = new SimpleDataSet(metadata());
    +            result.row(keyspace)
    +                  .column(CQL, SchemaCQLHelper.getKeyspaceAsCQL(keyspace));
    +            return result;
    +        }
    +
    +        public DataSet data()
    +        {
    +            SimpleDataSet result = new SimpleDataSet(metadata());
    +            for (String keyspace : Schema.instance.getKeyspaces())
    +            {
    +                result.row(keyspace)
    +                      .column(CQL, SchemaCQLHelper.getKeyspaceAsCQL(keyspace));
    +            }
    +            return result;
    +        }
    +    }
    +
    +    static abstract class AbstractDescribeTable extends AbstractVirtualTable
    +    {
    +        AbstractDescribeTable(String keyspace, String name)
    +        {
    +            super(TableMetadata.builder(keyspace, "describe_" + name)
    --- End diff --
    
    complexity there is there are things like having a UDT with same name as a table in the same keyspace. We can have something like ((type), keyspace, name) so that way we can provide uniqueness to everything and it would only be funny for keyspaces.


---

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