Skip to content

Commit

Permalink
added information_schema.schemata listing all available table schemas
Browse files Browse the repository at this point in the history
  • Loading branch information
msbt committed Dec 10, 2014
1 parent 59ef25c commit 14215c0
Show file tree
Hide file tree
Showing 8 changed files with 190 additions and 48 deletions.
3 changes: 3 additions & 0 deletions CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,9 @@ Changes for Crate
Unreleased
==========

- implemented table ``information_schema.schemata`` with column ``schema_name``
listing all available schemas

- Array types are now fully supported in dynamic schemas. Note that
Crate performs an internal migration, therefore downgrading to an
older version is not supported
Expand Down
20 changes: 19 additions & 1 deletion docs/sql/information_schema.txt
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ settings like the ``number of shards`` or ``number of replicas``::
| doc | quotes | 2 | 0 |
| information_schema | columns | 1 | 0 |
| information_schema | routines | 1 | 0 |
| information_schema | schemata | 1 | 0 |
| information_schema | table_constraints | 1 | 0 |
| information_schema | table_partitions | 1 | 0 |
| information_schema | tables | 1 | 0 |
Expand All @@ -44,7 +45,7 @@ settings like the ``number of shards`` or ``number of replicas``::
| sys | operations_log | 1 | 0 |
| sys | shards | 1 | 0 |
+--------------------+-------------------+------------------+--------------------+
SELECT 17 rows in set (... sec)
SELECT 18 rows in set (... sec)

The table also contains additional information such as specified routing
(:ref:`sql_ddl_sharding`) and partitioned by (:ref:`sql_ddl_partitioned_by`)
Expand Down Expand Up @@ -256,3 +257,20 @@ Or get an overview of how many routines and routine types are available::
| 54 | TOKEN_FILTER |
+----------+--------------+
SELECT 4 rows in set (... sec)

Schemata
========

The schemata table lists all existing schemas. These schemas are always
available: ``blob``, ``doc``, ``information_schema`` and ``sys``::

cr> select schema_name from information_schema.schemata order by schema_name;
+--------------------+
| schema_name |
+--------------------+
| blob |
| doc |
| information_schema |
| sys |
+--------------------+
SELECT 4 rows in set (... sec)
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ public InformationSchemaInfo() {
.put(InformationPartitionsTableInfo.NAME, new InformationPartitionsTableInfo(this))
.put(InformationTableConstraintsTableInfo.NAME, new InformationTableConstraintsTableInfo(this))
.put(InformationRoutinesTableInfo.NAME, new InformationRoutinesTableInfo(this))
.put(InformationSchemataTableInfo.NAME, new InformationSchemataTableInfo(this))
.build();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* Licensed to CRATE Technology GmbH ("Crate") under one or more contributor
* license agreements. See the NOTICE file distributed with this work for
* additional information regarding copyright ownership. Crate 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.
*
* However, if you have executed another commercial license agreement
* with Crate these terms will supersede the license and you may use the
* software solely pursuant to the terms of the relevant commercial agreement.
*/

package io.crate.metadata.information;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.crate.metadata.ColumnIdent;
import io.crate.metadata.ReferenceIdent;
import io.crate.metadata.ReferenceInfo;
import io.crate.metadata.TableIdent;
import io.crate.planner.RowGranularity;
import io.crate.types.DataType;
import io.crate.types.DataTypes;

public class InformationSchemataTableInfo extends InformationTableInfo {

public static final String NAME = "schemata";
public static final TableIdent IDENT = new TableIdent(InformationSchemaInfo.NAME, NAME);

public static class Columns {
public static final ColumnIdent SCHEMA_NAME = new ColumnIdent("schema_name");
}

public static class ReferenceInfos {
public static final ReferenceInfo SCHEMA_NAME = info(Columns.SCHEMA_NAME, DataTypes.STRING);
}

private static ReferenceInfo info(ColumnIdent columnIdent, DataType dataType) {
return new ReferenceInfo(new ReferenceIdent(IDENT, columnIdent), RowGranularity.DOC, dataType);
}

protected InformationSchemataTableInfo(InformationSchemaInfo schemaInfo) {
super(schemaInfo, IDENT,
ImmutableList.of(Columns.SCHEMA_NAME),
ImmutableMap.of(Columns.SCHEMA_NAME, ReferenceInfos.SCHEMA_NAME),
ImmutableList.of(ReferenceInfos.SCHEMA_NAME)
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,16 +47,9 @@

public class InformationSchemaCollectService implements CollectService {

private final Iterable<TableInfo> tablesIterable;
private final Iterable<ColumnContext> columnsIterable;
private final Iterable<TableInfo> tableConstraintsIterable;
private final Iterable<RoutineInfo> routinesIterable;

private final CollectInputSymbolVisitor<RowCollectExpression<?, ?>> docInputSymbolVisitor;
private final ImmutableMap<String, Iterable<?>> iterables;

private final Iterable<TablePartitionInfo> tablePartitionsIterable;

@Inject
protected InformationSchemaCollectService(Functions functions,
ReferenceInfos referenceInfos,
Expand All @@ -66,7 +59,7 @@ protected InformationSchemaCollectService(Functions functions,
RoutineInfos routineInfos = new RoutineInfos(ftResolver);
this.docInputSymbolVisitor = new CollectInputSymbolVisitor<>(functions, refResolver);

tablesIterable = FluentIterable.from(referenceInfos)
Iterable<TableInfo> tablesIterable = FluentIterable.from(referenceInfos)
.transformAndConcat(new Function<SchemaInfo, Iterable<TableInfo>>() {
@Nullable
@Override
Expand All @@ -82,8 +75,8 @@ public boolean apply(TableInfo input) {
});
}
});
tablePartitionsIterable = FluentIterable.from(new TablePartitionInfos(tablesIterable));
columnsIterable = FluentIterable
Iterable<TablePartitionInfo> tablePartitionsIterable = FluentIterable.from(new TablePartitionInfos(tablesIterable));
Iterable<ColumnContext> columnsIterable = FluentIterable
.from(tablesIterable)
.transformAndConcat(new Function<TableInfo, Iterable<ColumnContext>>() {
@Nullable
Expand All @@ -93,26 +86,26 @@ public Iterable<ColumnContext> apply(TableInfo input) {
return new ColumnsIterator(input);
}
});
tableConstraintsIterable = FluentIterable.from(tablesIterable).filter(new Predicate<TableInfo>() {
@Override
public boolean apply(@Nullable TableInfo input) {
return input != null && input.primaryKey().size() > 0;
}
Iterable<TableInfo> tableConstraintsIterable = FluentIterable.from(tablesIterable).filter(new Predicate<TableInfo>() {
@Override
public boolean apply(@Nullable TableInfo input) {
return input != null && input.primaryKey().size() > 0;
}
});
routinesIterable = FluentIterable.from(routineInfos)
Iterable<RoutineInfo> routinesIterable = FluentIterable.from(routineInfos)
.filter(new Predicate<RoutineInfo>() {
@Override
public boolean apply(@Nullable RoutineInfo input) {
return input != null;
}
});
this.iterables = ImmutableMap.of(
"information_schema.tables", tablesIterable,
"information_schema.columns", columnsIterable,
"information_schema.table_constraints", tableConstraintsIterable,
"information_schema.table_partitions", tablePartitionsIterable,
"information_schema.routines", routinesIterable
);
this.iterables = ImmutableMap.<String, Iterable<?>>builder()
.put("information_schema.tables", tablesIterable)
.put("information_schema.columns", columnsIterable)
.put("information_schema.table_constraints", tableConstraintsIterable)
.put("information_schema.table_partitions", tablePartitionsIterable)
.put("information_schema.routines", routinesIterable)
.put("information_schema.schemata", referenceInfos).build();
}

class ColumnsIterator implements Iterator<ColumnContext>, Iterable<ColumnContext> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,9 @@ public InformationDocLevelReferenceResolver() {
add(builder, InformationRoutinesExpression.ROUTINE_NAME_EXPRESSION);
add(builder, InformationRoutinesExpression.ROUTINE_TYPE_EXPRESSION);

// information_schema.schemata
add(builder, InformationSchemataExpression.SCHEMA_NAME_EXPRESSION);

implementations = builder.build();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* Licensed to CRATE Technology GmbH ("Crate") under one or more contributor
* license agreements. See the NOTICE file distributed with this work for
* additional information regarding copyright ownership. Crate 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.
*
* However, if you have executed another commercial license agreement
* with Crate these terms will supersede the license and you may use the
* software solely pursuant to the terms of the relevant commercial agreement.
*/

package io.crate.operation.reference.information;

import io.crate.metadata.ReferenceInfo;
import io.crate.metadata.information.InformationSchemataTableInfo;
import io.crate.metadata.information.RowCollectExpression;
import io.crate.metadata.table.SchemaInfo;
import org.apache.lucene.util.BytesRef;

public abstract class InformationSchemataExpression<T> extends RowCollectExpression<SchemaInfo, T> {

public static final SchemataSchemaNameExpression SCHEMA_NAME_EXPRESSION = new SchemataSchemaNameExpression();

protected InformationSchemataExpression(ReferenceInfo info) {
super(info);
}

private static class SchemataSchemaNameExpression extends InformationSchemataExpression<BytesRef> {

protected SchemataSchemaNameExpression() {
super(InformationSchemataTableInfo.ReferenceInfos.SCHEMA_NAME);
}

@Override
public BytesRef value() {
return new BytesRef(this.row.name());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import io.crate.test.integration.CrateIntegrationTest;
import io.crate.testing.TestingHelpers;
import org.elasticsearch.common.collect.MapBuilder;
import org.hamcrest.Matchers;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
Expand Down Expand Up @@ -64,20 +65,21 @@ private void serviceSetup() {
@Test
public void testDefaultTables() throws Exception {
execute("select * from information_schema.tables order by schema_name, table_name");
assertEquals(12L, response.rowCount());
assertEquals(13L, response.rowCount());

assertArrayEquals(response.rows()[0], new Object[]{"information_schema", "columns", 1, "0", null, null, null});
assertArrayEquals(response.rows()[1], new Object[]{"information_schema", "routines", 1, "0", null, null, null});
assertArrayEquals(response.rows()[2], new Object[]{"information_schema", "table_constraints", 1, "0", null, null, null});
assertArrayEquals(response.rows()[3], new Object[]{"information_schema", "table_partitions", 1, "0", null, null, null});
assertArrayEquals(response.rows()[4], new Object[]{"information_schema", "tables", 1, "0", null, null, null});
assertArrayEquals(response.rows()[5], new Object[]{"sys", "cluster", 1, "0", null, null, null});
assertArrayEquals(response.rows()[6], new Object[]{"sys", "jobs", 1, "0", null, null, null});
assertArrayEquals(response.rows()[7], new Object[]{"sys", "jobs_log", 1, "0", null, null, null});
assertArrayEquals(response.rows()[8], new Object[]{"sys", "nodes", 1, "0", null, null, null});
assertArrayEquals(response.rows()[9], new Object[]{"sys", "operations", 1, "0", null, null, null});
assertArrayEquals(response.rows()[10], new Object[]{"sys", "operations_log", 1, "0", null, null, null});
assertArrayEquals(response.rows()[11], new Object[]{"sys", "shards", 1, "0", null, null, null});
assertArrayEquals(response.rows()[2], new Object[]{"information_schema", "schemata", 1, "0", null, null, null});
assertArrayEquals(response.rows()[3], new Object[]{"information_schema", "table_constraints", 1, "0", null, null, null});
assertArrayEquals(response.rows()[4], new Object[]{"information_schema", "table_partitions", 1, "0", null, null, null});
assertArrayEquals(response.rows()[5], new Object[]{"information_schema", "tables", 1, "0", null, null, null});
assertArrayEquals(response.rows()[6], new Object[]{"sys", "cluster", 1, "0", null, null, null});
assertArrayEquals(response.rows()[7], new Object[]{"sys", "jobs", 1, "0", null, null, null});
assertArrayEquals(response.rows()[8], new Object[]{"sys", "jobs_log", 1, "0", null, null, null});
assertArrayEquals(response.rows()[9], new Object[]{"sys", "nodes", 1, "0", null, null, null});
assertArrayEquals(response.rows()[10], new Object[]{"sys", "operations", 1, "0", null, null, null});
assertArrayEquals(response.rows()[11], new Object[]{"sys", "operations_log", 1, "0", null, null, null});
assertArrayEquals(response.rows()[12], new Object[]{"sys", "shards", 1, "0", null, null, null});
}

@Test
Expand Down Expand Up @@ -117,7 +119,7 @@ public void testSearchInformationSchemaTablesRefresh() throws Exception {
serviceSetup();

execute("select * from information_schema.tables");
assertEquals(15L, response.rowCount());
assertEquals(16L, response.rowCount());

client().execute(SQLAction.INSTANCE,
new SQLRequest("create table t4 (col1 integer, col2 string)")).actionGet();
Expand All @@ -127,7 +129,7 @@ public void testSearchInformationSchemaTablesRefresh() throws Exception {
Thread.sleep(10);

execute("select * from information_schema.tables");
assertEquals(16L, response.rowCount());
assertEquals(17L, response.rowCount());
}

@Test
Expand Down Expand Up @@ -262,21 +264,23 @@ public void testSelectStarFromInformationSchemaTable() throws Exception {
public void testSelectFromTableConstraints() throws Exception {

execute("select * from INFORMATION_SCHEMA.table_constraints order by schema_name asc, table_name asc");
assertEquals(6L, response.rowCount());
assertEquals(7L, response.rowCount());
assertThat(response.cols(), arrayContaining("schema_name", "table_name", "constraint_name",
"constraint_type"));
assertThat(dotJoiner.join(response.rows()[0][0], response.rows()[0][1]), is("information_schema.columns"));
assertThat(commaJoiner.join((String[]) response.rows()[0][2]), is("schema_name, table_name, column_name"));
assertThat(dotJoiner.join(response.rows()[1][0], response.rows()[1][1]), is("information_schema.tables"));
assertThat(commaJoiner.join((String[])response.rows()[1][2]), is("schema_name, table_name"));
assertThat(dotJoiner.join(response.rows()[2][0], response.rows()[2][1]), is("sys.jobs"));
assertThat(commaJoiner.join((String[])response.rows()[2][2]), is("id"));
assertThat(dotJoiner.join(response.rows()[3][0], response.rows()[3][1]), is("sys.jobs_log"));
assertThat(dotJoiner.join(response.rows()[1][0], response.rows()[1][1]), is("information_schema.schemata"));
assertThat(commaJoiner.join((String[])response.rows()[1][2]), is("schema_name"));
assertThat(dotJoiner.join(response.rows()[2][0], response.rows()[2][1]), is("information_schema.tables"));
assertThat(commaJoiner.join((String[])response.rows()[2][2]), is("schema_name, table_name"));
assertThat(dotJoiner.join(response.rows()[3][0], response.rows()[3][1]), is("sys.jobs"));
assertThat(commaJoiner.join((String[])response.rows()[3][2]), is("id"));
assertThat(dotJoiner.join(response.rows()[4][0], response.rows()[4][1]), is("sys.nodes"));
assertThat(dotJoiner.join(response.rows()[4][0], response.rows()[4][1]), is("sys.jobs_log"));
assertThat(commaJoiner.join((String[])response.rows()[4][2]), is("id"));
assertThat(dotJoiner.join(response.rows()[5][0], response.rows()[5][1]), is("sys.shards"));
assertThat(commaJoiner.join((String[])response.rows()[5][2]), is("schema_name, table_name, id, partition_ident"));
assertThat(dotJoiner.join(response.rows()[5][0], response.rows()[5][1]), is("sys.nodes"));
assertThat(commaJoiner.join((String[])response.rows()[5][2]), is("id"));
assertThat(dotJoiner.join(response.rows()[6][0], response.rows()[6][1]), is("sys.shards"));
assertThat(commaJoiner.join((String[])response.rows()[6][2]), is("schema_name, table_name, id, partition_ident"));

execute("create table test (col1 integer primary key, col2 string)");
ensureGreen();
Expand Down Expand Up @@ -427,7 +431,7 @@ public void testTableConstraintsWithOrderBy() throws Exception {
@Test
public void testDefaultColumns() throws Exception {
execute("select * from information_schema.columns order by schema_name, table_name");
assertEquals(200L, response.rowCount());
assertEquals(201L, response.rowCount());
}

@Test
Expand Down Expand Up @@ -612,7 +616,7 @@ public void testGlobalCount() throws Exception {
ensureYellow();
execute("select count(*) from information_schema.tables");
assertEquals(1, response.rowCount());
assertEquals(15L, response.rows()[0][0]);
assertEquals(16L, response.rows()[0][0]);
}

@Test
Expand Down Expand Up @@ -931,4 +935,17 @@ public void testRegexpMatch() throws Exception {
assertThat((String)response.rows()[1][0], is("doc"));
}

@Test
public void testSelectSchemata() throws Exception {
execute("select * from information_schema.schemata order by schema_name asc");
assertThat(response.rowCount(), is(4L));
assertThat(TestingHelpers.getColumn(response.rows(), 0), is(Matchers.<Object>arrayContaining("blob", "doc", "information_schema", "sys")));

execute("create table t1 (col string) with (number_of_replicas=0)");
ensureGreen();

execute("select * from information_schema.schemata order by schema_name asc");
assertThat(response.rowCount(), is(4L));
assertThat(TestingHelpers.getColumn(response.rows(), 0), is(Matchers.<Object>arrayContaining("blob", "doc", "information_schema", "sys")));
}
}

0 comments on commit 14215c0

Please sign in to comment.