diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java index 5462f86a209..61fb52c0423 100644 --- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java +++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java @@ -29,6 +29,7 @@ import org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractSchema; import org.apache.calcite.schema.impl.MaterializedViewTable; +import org.apache.calcite.schema.lookup.LikePattern; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.SqlWriterConfig; @@ -325,8 +326,8 @@ private void addMaterializedViews() { query = buf.toString(); // Add the view for this query - String viewName = "$" + getTableNames().size(); - SchemaPlus schema = parentSchema.getSubSchema(name); + String viewName = "$" + tables().getNames(LikePattern.any()).size(); + SchemaPlus schema = parentSchema.subSchemas().get(name); if (schema == null) { throw new IllegalStateException("Cannot find schema " + name + " in parent schema " + parentSchema.getName()); diff --git a/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java b/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java index cdada3cfe04..87cd8666ba9 100644 --- a/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java +++ b/core/src/main/java/org/apache/calcite/adapter/clone/CloneSchema.java @@ -33,6 +33,8 @@ import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractSchema; +import org.apache.calcite.schema.lookup.LikePattern; +import org.apache.calcite.schema.lookup.Lookup; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; @@ -68,8 +70,9 @@ public CloneSchema(SchemaPlus sourceSchema) { @Override protected Map getTableMap() { final Map map = new LinkedHashMap<>(); - for (String name : sourceSchema.getTableNames()) { - final Table table = sourceSchema.getTable(name); + final Lookup tables = sourceSchema.tables(); + for (String name : tables.getNames(LikePattern.any())) { + final Table table = tables.get(name); if (table instanceof QueryableTable) { final QueryableTable sourceTable = (QueryableTable) table; map.put(name, diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcBaseSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcBaseSchema.java new file mode 100644 index 00000000000..205e6fed416 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcBaseSchema.java @@ -0,0 +1,94 @@ +/* + * 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.calcite.adapter.jdbc; + +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.schema.Function; +import org.apache.calcite.schema.Schema; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.SchemaVersion; +import org.apache.calcite.schema.Schemas; +import org.apache.calcite.schema.Table; +import org.apache.calcite.schema.lookup.LikePattern; +import org.apache.calcite.schema.lookup.Lookup; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Collection; +import java.util.Collections; +import java.util.Set; + +import static java.util.Objects.requireNonNull; + +/** + * Base class for JDBC schemas. + */ +public abstract class JdbcBaseSchema implements Schema { + + @Override public abstract Lookup
tables(); + + + @Override public @Nullable Table getTable(String name) { + return tables().get(name); + } + + @Override public Set getTableNames() { + return tables().getNames(LikePattern.any()); + } + + @Override public abstract Lookup subSchemas(); + + @Override public @Nullable Schema getSubSchema(String name) { + return subSchemas().get(name); + } + + @Override public Set getSubSchemaNames() { + return subSchemas().getNames(LikePattern.any()); + } + + + @Override public @Nullable RelProtoDataType getType(String name) { + return null; + } + + @Override public Set getTypeNames() { + return Collections.emptySet(); + } + + @Override public final Collection getFunctions(String name) { + return Collections.emptyList(); + } + + @Override public final Set getFunctionNames() { + return Collections.emptySet(); + } + + @Override public Expression getExpression(final @Nullable SchemaPlus parentSchema, + final String name) { + requireNonNull(parentSchema, "parentSchema"); + return Schemas.subSchemaExpression(parentSchema, name, getClass()); + } + + @Override public boolean isMutable() { + return false; + } + + @Override public Schema snapshot(final SchemaVersion version) { + return this; + } +} diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcCatalogSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcCatalogSchema.java index ba63e0f4946..54330d327fa 100644 --- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcCatalogSchema.java +++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcCatalogSchema.java @@ -22,22 +22,27 @@ import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.Schemas; +import org.apache.calcite.schema.Table; import org.apache.calcite.schema.Wrapper; -import org.apache.calcite.schema.impl.AbstractSchema; +import org.apache.calcite.schema.lookup.IgnoreCaseLookup; +import org.apache.calcite.schema.lookup.LikePattern; +import org.apache.calcite.schema.lookup.LoadingCacheLookup; +import org.apache.calcite.schema.lookup.Lookup; import org.apache.calcite.sql.SqlDialect; import org.apache.calcite.sql.SqlDialectFactory; import org.apache.calcite.sql.SqlDialectFactoryImpl; import org.apache.calcite.util.BuiltInMethod; import com.google.common.base.Suppliers; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; -import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.function.Supplier; import javax.sql.DataSource; @@ -51,20 +56,21 @@ * an instance of {@link JdbcSchema}. * *

This schema is lazy: it does not compute the list of schema names until - * the first call to {@link #getSubSchemaMap()}. Then it creates a - * {@link JdbcSchema} for each schema name. Each JdbcSchema will populate its + * the first call to {@link #subSchemas()} and {@link Lookup#get(String)}. Then it creates a + * {@link JdbcSchema} for this schema name. Each JdbcSchema will populate its * tables on demand. */ -public class JdbcCatalogSchema extends AbstractSchema implements Wrapper { +public class JdbcCatalogSchema extends JdbcBaseSchema implements Wrapper { final DataSource dataSource; public final SqlDialect dialect; final JdbcConvention convention; final String catalog; + private final Lookup subSchemas; - /** Sub-schemas by name, lazily initialized. */ + /** default schema name, lazily initialized. */ @SuppressWarnings({"method.invocation.invalid", "Convert2MethodRef"}) - final Supplier subSchemaMapSupplier = - Suppliers.memoize(() -> computeSubSchemaMap()); + private final Supplier> defaultSchemaName = + Suppliers.memoize(() -> Optional.ofNullable(computeDefaultSchemaName())); /** Creates a JdbcCatalogSchema. */ public JdbcCatalogSchema(DataSource dataSource, SqlDialect dialect, @@ -73,6 +79,40 @@ public JdbcCatalogSchema(DataSource dataSource, SqlDialect dialect, this.dialect = requireNonNull(dialect, "dialect"); this.convention = requireNonNull(convention, "convention"); this.catalog = catalog; + this.subSchemas = new LoadingCacheLookup<>(new IgnoreCaseLookup() { + @Override public @Nullable JdbcSchema get(String name) { + try (Connection connection = dataSource.getConnection(); + ResultSet resultSet = + connection.getMetaData().getSchemas(catalog, name)) { + while (resultSet.next()) { + final String schemaName = + requireNonNull(resultSet.getString(1), + "got null schemaName from the database"); + return new JdbcSchema(dataSource, dialect, convention, catalog, schemaName); + } + } catch (SQLException e) { + throw new RuntimeException(e); + } + return null; + } + + @Override public Set getNames(LikePattern pattern) { + final ImmutableSet.Builder builder = + ImmutableSet.builder(); + try (Connection connection = dataSource.getConnection(); + ResultSet resultSet = + connection.getMetaData().getSchemas(catalog, pattern.pattern)) { + while (resultSet.next()) { + builder.add( + requireNonNull(resultSet.getString(1), + "got null schemaName from the database")); + } + } catch (SQLException e) { + throw new RuntimeException(e); + } + return builder.build(); + } + }); } public static JdbcCatalogSchema create( @@ -103,34 +143,25 @@ public static JdbcCatalogSchema create( return new JdbcCatalogSchema(dataSource, dialect, convention, catalog); } - private SubSchemaMap computeSubSchemaMap() { - final ImmutableMap.Builder builder = - ImmutableMap.builder(); - @Nullable String defaultSchemaName; - try (Connection connection = dataSource.getConnection(); - ResultSet resultSet = - connection.getMetaData().getSchemas(catalog, null)) { - defaultSchemaName = connection.getSchema(); - while (resultSet.next()) { - final String schemaName = - requireNonNull(resultSet.getString(1), - "got null schemaName from the database"); - builder.put(schemaName, - new JdbcSchema(dataSource, dialect, convention, catalog, schemaName)); - } + @Override public Lookup

tables() { + return Lookup.empty(); + } + + @Override public Lookup subSchemas() { + return subSchemas; + } + + private @Nullable String computeDefaultSchemaName() { + try (Connection connection = dataSource.getConnection()) { + return connection.getSchema(); } catch (SQLException e) { throw new RuntimeException(e); } - return new SubSchemaMap(defaultSchemaName, builder.build()); - } - - @Override protected Map getSubSchemaMap() { - return subSchemaMapSupplier.get().map; } /** Returns the name of the default sub-schema. */ public @Nullable String getDefaultSubSchemaName() { - return subSchemaMapSupplier.get().defaultSchemaName; + return defaultSchemaName.get().orElse(null); } /** Returns the data source. */ @@ -148,16 +179,4 @@ public DataSource getDataSource() { } return null; } - - /** Contains sub-schemas by name, and the name of the default schema. */ - private static class SubSchemaMap { - final @Nullable String defaultSchemaName; - final ImmutableMap map; - - private SubSchemaMap(@Nullable String defaultSchemaName, - ImmutableMap map) { - this.defaultSchemaName = defaultSchemaName; - this.map = map; - } - } } diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java index 963f6bf5e7c..64b64d16a54 100644 --- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java +++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java @@ -26,7 +26,6 @@ import org.apache.calcite.rel.type.RelDataTypeImpl; import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.calcite.rel.type.RelProtoDataType; -import org.apache.calcite.schema.Function; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaFactory; import org.apache.calcite.schema.SchemaPlus; @@ -34,19 +33,19 @@ import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; import org.apache.calcite.schema.Wrapper; +import org.apache.calcite.schema.lookup.IgnoreCaseLookup; +import org.apache.calcite.schema.lookup.LikePattern; +import org.apache.calcite.schema.lookup.Lookup; import org.apache.calcite.sql.SqlDialect; import org.apache.calcite.sql.SqlDialectFactory; import org.apache.calcite.sql.SqlDialectFactoryImpl; import org.apache.calcite.sql.type.SqlTypeFactoryImpl; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.util.LazyReference; import org.apache.calcite.util.Pair; import org.apache.calcite.util.Util; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableMultimap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; import org.checkerframework.checker.nullness.qual.Nullable; @@ -58,13 +57,17 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; -import java.util.ArrayList; -import java.util.Collection; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; import javax.sql.DataSource; import static java.lang.Integer.parseInt; @@ -77,7 +80,7 @@ * queries against this schema are executed against those tables, pushing down * as much as possible of the query logic to SQL. */ -public class JdbcSchema implements Schema, Wrapper { +public class JdbcSchema extends JdbcBaseSchema implements Schema, Wrapper { private static final Logger LOGGER = LoggerFactory.getLogger(JdbcSchema.class); final DataSource dataSource; @@ -85,8 +88,8 @@ public class JdbcSchema implements Schema, Wrapper { final @Nullable String schema; public final SqlDialect dialect; final JdbcConvention convention; - private @Nullable ImmutableMap tableMap; - private final boolean snapshot; + private final LazyReference> tables = new LazyReference<>(); + private final Lookup subSchemas = Lookup.empty(); @Experimental public static final ThreadLocal<@Nullable Foo> THREAD_METADATA = new ThreadLocal<>(); @@ -105,19 +108,11 @@ public class JdbcSchema implements Schema, Wrapper { */ public JdbcSchema(DataSource dataSource, SqlDialect dialect, JdbcConvention convention, @Nullable String catalog, @Nullable String schema) { - this(dataSource, dialect, convention, catalog, schema, null); - } - - private JdbcSchema(DataSource dataSource, SqlDialect dialect, - JdbcConvention convention, @Nullable String catalog, @Nullable String schema, - @Nullable ImmutableMap tableMap) { this.dataSource = requireNonNull(dataSource, "dataSource"); this.dialect = requireNonNull(dialect, "dialect"); this.convention = convention; this.catalog = catalog; this.schema = schema; - this.tableMap = tableMap; - this.snapshot = tableMap != null; } public static JdbcSchema create( @@ -218,13 +213,33 @@ public static DataSource dataSource(String url, @Nullable String driverClassName password); } + @Override public Lookup
tables() { + return tables.getOrCompute(() -> new IgnoreCaseLookup
() { + @Override public @Nullable Table get(String name) { + try (Stream s = getMetaTableStream(name)) { + return s.findFirst().map(it -> jdbcTableMapper(it)).orElse(null); + } + } + + @Override public Set getNames(LikePattern pattern) { + try (Stream s = getMetaTableStream(pattern.pattern)) { + return s.map(it -> it.tableName).collect(Collectors.toSet()); + } + } + }); + } + + @Override public Lookup subSchemas() { + return subSchemas; + } + + @Override public boolean isMutable() { return false; } @Override public Schema snapshot(SchemaVersion version) { - return new JdbcSchema(dataSource, dialect, convention, catalog, schema, - tableMap); + return this; } // Used by generated code. @@ -237,80 +252,86 @@ public DataSource getDataSource() { return Schemas.subSchemaExpression(parentSchema, name, JdbcSchema.class); } - protected Multimap getFunctions() { - // TODO: populate map from JDBC metadata - return ImmutableMultimap.of(); + private Stream getMetaTableStream(String tableNamePattern) { + final Pair<@Nullable String, @Nullable String> catalogSchema = getCatalogSchema(); + final Stream tableDefs; + Connection connection = null; + ResultSet resultSet = null; + try { + connection = dataSource.getConnection(); + final DatabaseMetaData metaData = connection.getMetaData(); + resultSet = + metaData.getTables(catalogSchema.left, catalogSchema.right, tableNamePattern, null); + tableDefs = asStream(connection, resultSet) + .map(JdbcSchema::metaDataMapper); + } catch (SQLException e) { + close(connection, null, resultSet); + throw new RuntimeException( + "Exception while reading tables", e); + } + return tableDefs; } - @Override public final Collection getFunctions(String name) { - return getFunctions().get(name); // never null + private static Stream asStream(Connection connection, ResultSet resultSet) { + return StreamSupport.stream( + new Spliterators.AbstractSpliterator( + Long.MAX_VALUE, Spliterator.ORDERED) { + @Override public boolean tryAdvance(Consumer action) { + try { + if (!resultSet.next()) { + return false; + } + action.accept(resultSet); + return true; + } catch (SQLException ex) { + throw new RuntimeException(ex); + } + } + }, false).onClose(() -> close(connection, null, resultSet)); } - @Override public final Set getFunctionNames() { - return getFunctions().keySet(); + private JdbcTable jdbcTableMapper(MetaImpl.MetaTable tableDef) { + return new JdbcTable(this, tableDef.tableCat, tableDef.tableSchem, tableDef.tableName, + getTableType(tableDef.tableType)); } - private ImmutableMap computeTables() { - Connection connection = null; - ResultSet resultSet = null; + private static MetaImpl.MetaTable metaDataMapper(ResultSet resultSet) { try { - connection = dataSource.getConnection(); - final Pair<@Nullable String, @Nullable String> catalogSchema = getCatalogSchema(connection); - final String catalog = catalogSchema.left; - final String schema = catalogSchema.right; - final Iterable tableDefs; - Foo threadMetadata = THREAD_METADATA.get(); - if (threadMetadata != null) { - tableDefs = threadMetadata.apply(catalog, schema); - } else { - final List tableDefList = new ArrayList<>(); - final DatabaseMetaData metaData = connection.getMetaData(); - resultSet = metaData.getTables(catalog, schema, null, null); - while (resultSet.next()) { - final String catalogName = resultSet.getString(1); - final String schemaName = resultSet.getString(2); - final String tableName = resultSet.getString(3); - final String tableTypeName = resultSet.getString(4); - tableDefList.add( - new MetaImpl.MetaTable(catalogName, schemaName, tableName, - tableTypeName)); - } - tableDefs = tableDefList; - } + return new MetaImpl.MetaTable(intern(resultSet.getString(1)), intern(resultSet.getString(2)), + intern(resultSet.getString(3)), + intern(resultSet.getString(4))); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } - final ImmutableMap.Builder builder = - ImmutableMap.builder(); - for (MetaImpl.MetaTable tableDef : tableDefs) { - // Clean up table type. In particular, this ensures that 'SYSTEM TABLE', - // returned by Phoenix among others, maps to TableType.SYSTEM_TABLE. - // We know enum constants are upper-case without spaces, so we can't - // make things worse. - // - // PostgreSQL returns tableTypeName==null for pg_toast* tables - // This can happen if you start JdbcSchema off a "public" PG schema - // The tables are not designed to be queried by users, however we do - // not filter them as we keep all the other table types. - final String tableTypeName2 = - tableDef.tableType == null + private static @Nullable String intern(@Nullable String string) { + if (string == null) { + return null; + } + return string.intern(); + } + + private static TableType getTableType(String tableTypeName) { + // Clean up table type. In particular, this ensures that 'SYSTEM TABLE', + // returned by Phoenix among others, maps to TableType.SYSTEM_TABLE. + // We know enum constants are upper-case without spaces, so we can't + // make things worse. + // + // PostgreSQL returns tableTypeName==null for pg_toast* tables + // This can happen if you start JdbcSchema off a "public" PG schema + // The tables are not designed to be queried by users, however we do + // not filter them as we keep all the other table types. + final String tableTypeName2 = + tableTypeName == null ? null - : tableDef.tableType.toUpperCase(Locale.ROOT).replace(' ', '_'); - final TableType tableType = - Util.enumVal(TableType.OTHER, tableTypeName2); - if (tableType == TableType.OTHER && tableTypeName2 != null) { - LOGGER.info("Unknown table type: {}", tableTypeName2); - } - final JdbcTable table = - new JdbcTable(this, tableDef.tableCat, tableDef.tableSchem, - tableDef.tableName, tableType); - builder.put(tableDef.tableName, table); - } - return builder.build(); - } catch (SQLException e) { - throw new RuntimeException( - "Exception while reading tables", e); - } finally { - close(connection, null, resultSet); + : tableTypeName.toUpperCase(Locale.ROOT).replace(' ', '_'); + final TableType tableType = + Util.enumVal(TableType.OTHER, tableTypeName2); + if (tableType == TableType.OTHER && tableTypeName2 != null) { + LOGGER.info("Unknown table type: {}", tableTypeName2); } + return tableType; } /** Returns [major, minor] version from a database metadata. */ @@ -320,49 +341,40 @@ private static List version(DatabaseMetaData metaData) throws SQLExcept } /** Returns a pair of (catalog, schema) for the current connection. */ - private Pair<@Nullable String, @Nullable String> getCatalogSchema(Connection connection) - throws SQLException { - final DatabaseMetaData metaData = connection.getMetaData(); - final List version41 = ImmutableList.of(4, 1); // JDBC 4.1 - String catalog = this.catalog; - String schema = this.schema; - final boolean jdbc41OrAbove = - VERSION_ORDERING.compare(version(metaData), version41) >= 0; - if (catalog == null && jdbc41OrAbove) { - // From JDBC 4.1, catalog and schema can be retrieved from the connection - // object, hence try to get it from there if it was not specified by user - catalog = connection.getCatalog(); - } - if (schema == null && jdbc41OrAbove) { - schema = connection.getSchema(); - if ("".equals(schema)) { - schema = null; // PostgreSQL returns useless "" sometimes + private Pair<@Nullable String, @Nullable String> getCatalogSchema() { + try (Connection connection = dataSource.getConnection()) { + final DatabaseMetaData metaData = connection.getMetaData(); + final List version41 = ImmutableList.of(4, 1); // JDBC 4.1 + String catalog = this.catalog; + String schema = this.schema; + final boolean jdbc41OrAbove = + VERSION_ORDERING.compare(version(metaData), version41) >= 0; + if (catalog == null && jdbc41OrAbove) { + // From JDBC 4.1, catalog and schema can be retrieved from the connection + // object, hence try to get it from there if it was not specified by user + catalog = connection.getCatalog(); } - } - if ((catalog == null || schema == null) - && metaData.getDatabaseProductName().equals("PostgreSQL")) { - final String sql = "select current_database(), current_schema()"; - try (Statement statement = connection.createStatement(); - ResultSet resultSet = statement.executeQuery(sql)) { - if (resultSet.next()) { - catalog = resultSet.getString(1); - schema = resultSet.getString(2); + if (schema == null && jdbc41OrAbove) { + schema = connection.getSchema(); + if ("".equals(schema)) { + schema = null; // PostgreSQL returns useless "" sometimes } } + if ((catalog == null || schema == null) + && metaData.getDatabaseProductName().equals("PostgreSQL")) { + final String sql = "select current_database(), current_schema()"; + try (Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(sql)) { + if (resultSet.next()) { + catalog = resultSet.getString(1); + schema = resultSet.getString(2); + } + } + } + return Pair.of(catalog, schema); + } catch (SQLException e) { + throw new RuntimeException(e); } - return Pair.of(catalog, schema); - } - - @Override public @Nullable Table getTable(String name) { - return getTableMap(false).get(name); - } - - private synchronized ImmutableMap getTableMap( - boolean force) { - if (force || tableMap == null) { - tableMap = computeTables(); - } - return tableMap; } RelProtoDataType getRelDataType(String catalogName, String schemaName, @@ -485,35 +497,6 @@ private static RelDataType parseTypeString(RelDataTypeFactory typeFactory, } } - @Override public Set getTableNames() { - // This method is called during a cache refresh. We can take it as a signal - // that we need to re-build our own cache. - return getTableMap(!snapshot).keySet(); - } - - protected Map getTypes() { - // TODO: populate map from JDBC metadata - return ImmutableMap.of(); - } - - @Override public @Nullable RelProtoDataType getType(String name) { - return getTypes().get(name); - } - - @Override public Set getTypeNames() { - //noinspection RedundantCast - return (Set) getTypes().keySet(); - } - - @Override public @Nullable Schema getSubSchema(String name) { - // JDBC does not support sub-schemas. - return null; - } - - @Override public Set getSubSchemaNames() { - return ImmutableSet.of(); - } - @Override public @Nullable T unwrap(Class clazz) { if (clazz.isInstance(this)) { return clazz.cast(this); diff --git a/core/src/main/java/org/apache/calcite/interpreter/TableScanNode.java b/core/src/main/java/org/apache/calcite/interpreter/TableScanNode.java index 51dbbdd02db..98ece3e24f2 100644 --- a/core/src/main/java/org/apache/calcite/interpreter/TableScanNode.java +++ b/core/src/main/java/org/apache/calcite/interpreter/TableScanNode.java @@ -131,7 +131,7 @@ private static TableScanNode createQueryable(Compiler compiler, requireNonNull(schema, () -> "schema is null while resolving " + name + " for table" + relOptTable.getQualifiedName()); - schema = schema.getSubSchema(name); + schema = schema.subSchemas().get(name); } final Enumerable rowEnumerable; if (elementType instanceof Class) { diff --git a/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java b/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java index 19b204aafa0..265be58dab8 100644 --- a/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java +++ b/core/src/main/java/org/apache/calcite/jdbc/CachingCalciteSchema.java @@ -22,13 +22,12 @@ import org.apache.calcite.schema.SchemaVersion; import org.apache.calcite.schema.Table; import org.apache.calcite.schema.TableMacro; +import org.apache.calcite.schema.lookup.CachedLookup; +import org.apache.calcite.schema.lookup.Lookup; import org.apache.calcite.util.NameMap; import org.apache.calcite.util.NameMultimap; import org.apache.calcite.util.NameSet; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.ImmutableSortedSet; @@ -37,7 +36,7 @@ import java.util.Collection; import java.util.List; -import java.util.Set; +import java.util.concurrent.ConcurrentLinkedDeque; import static org.apache.calcite.linq4j.Nullness.castNonNull; @@ -46,8 +45,7 @@ * functions and sub-schemas. */ class CachingCalciteSchema extends CalciteSchema { - private final Cached implicitSubSchemaCache; - private final Cached implicitTableCache; + private final ConcurrentLinkedDeque> caches = new ConcurrentLinkedDeque<>(); private final Cached implicitFunctionCache; private final Cached implicitTypeCache; @@ -72,20 +70,6 @@ private CachingCalciteSchema(@Nullable CalciteSchema parent, Schema schema, @Nullable List> path) { super(parent, schema, name, subSchemaMap, tableMap, latticeMap, typeMap, functionMap, functionNames, nullaryFunctionMap, path); - this.implicitSubSchemaCache = - new AbstractCached() { - @Override public SubSchemaCache build() { - return new SubSchemaCache(CachingCalciteSchema.this, - CachingCalciteSchema.this.schema.getSubSchemaNames()); - } - }; - this.implicitTableCache = - new AbstractCached() { - @Override public NameSet build() { - return NameSet.immutableCopyOf( - CachingCalciteSchema.this.schema.getTableNames()); - } - }; this.implicitFunctionCache = new AbstractCached() { @Override public NameSet build() { @@ -106,9 +90,8 @@ private CachingCalciteSchema(@Nullable CalciteSchema parent, Schema schema, if (cache == this.cache) { return; } + enableCaches(cache); final long now = System.currentTimeMillis(); - implicitSubSchemaCache.enable(now, cache); - implicitTableCache.enable(now, cache); implicitFunctionCache.enable(now, cache); this.cache = cache; } @@ -117,15 +100,14 @@ private CachingCalciteSchema(@Nullable CalciteSchema parent, Schema schema, return this.cache; } - @Override protected @Nullable CalciteSchema getImplicitSubSchema(String schemaName, - boolean caseSensitive) { - final long now = System.currentTimeMillis(); - final SubSchemaCache subSchemaCache = implicitSubSchemaCache.get(now); - for (String schemaName2 - : subSchemaCache.names.range(schemaName, caseSensitive)) { - return subSchemaCache.cache.getUnchecked(schemaName2); - } - return null; + @Override protected CalciteSchema createSubSchema(Schema schema, String name) { + return new CachingCalciteSchema(this, schema, name); + } + + @Override protected Lookup decorateLookup(Lookup lookup) { + CachedLookup cachedLookup = new CachedLookup<>(lookup); + caches.add(cachedLookup); + return cachedLookup; } /** Adds a child schema of this schema. */ @@ -136,20 +118,6 @@ private CachingCalciteSchema(@Nullable CalciteSchema parent, Schema schema, return calciteSchema; } - @Override protected @Nullable TableEntry getImplicitTable(String tableName, - boolean caseSensitive) { - final long now = System.currentTimeMillis(); - final NameSet implicitTableNames = implicitTableCache.get(now); - for (String tableName2 - : implicitTableNames.range(tableName, caseSensitive)) { - final Table table = schema.getTable(tableName2); - if (table != null) { - return tableEntry(tableName2, table); - } - } - return null; - } - @Override protected @Nullable TypeEntry getImplicitType(String name, boolean caseSensitive) { final long now = System.currentTimeMillis(); @@ -164,29 +132,6 @@ private CachingCalciteSchema(@Nullable CalciteSchema parent, Schema schema, return null; } - @Override protected void addImplicitSubSchemaToBuilder( - ImmutableSortedMap.Builder builder) { - ImmutableSortedMap explicitSubSchemas = - builder.build(); - final long now = System.currentTimeMillis(); - final SubSchemaCache subSchemaCache = implicitSubSchemaCache.get(now); - for (String name : subSchemaCache.names.iterable()) { - if (explicitSubSchemas.containsKey(name)) { - // explicit sub-schema wins. - continue; - } - builder.put(name, subSchemaCache.cache.getUnchecked(name)); - } - } - - @Override protected void addImplicitTableToBuilder( - ImmutableSortedSet.Builder builder) { - // Add implicit tables, case-sensitive. - final long now = System.currentTimeMillis(); - final NameSet set = implicitTableCache.get(now); - builder.addAll(set.iterable()); - } - @Override protected void addImplicitFunctionsToBuilder( ImmutableList.Builder builder, String name, boolean caseSensitive) { @@ -270,9 +215,8 @@ private CachingCalciteSchema(@Nullable CalciteSchema parent, Schema schema, @Override public boolean removeTable(String name) { if (cache) { - final long now = System.nanoTime(); - implicitTableCache.enable(now, false); - implicitTableCache.enable(now, true); + enableCaches(false); + enableCaches(true); } return super.removeTable(name); } @@ -286,6 +230,12 @@ private CachingCalciteSchema(@Nullable CalciteSchema parent, Schema schema, return super.removeFunction(name); } + private void enableCaches(final boolean cache) { + for (CachedLookup lookupCache : caches) { + lookupCache.enable(cache); + } + } + /** Strategy for caching the value of an object and re-creating it if its * value is out of date as of a given timestamp. * @@ -328,32 +278,4 @@ private abstract class AbstractCached implements Cached { built = false; } } - - /** Information about the implicit sub-schemas of an {@link CalciteSchema}. */ - private static class SubSchemaCache { - /** The names of sub-schemas returned from the {@link Schema} SPI. */ - final NameSet names; - /** Cached {@link CalciteSchema} wrappers. It is - * worth caching them because they contain maps of their own sub-objects. */ - final LoadingCache cache; - - private SubSchemaCache(final CalciteSchema calciteSchema, - Set names) { - this.names = NameSet.immutableCopyOf(names); - this.cache = - CacheBuilder.newBuilder() - .build(new CacheLoader() { - @Override public CalciteSchema load(String schemaName) { - final Schema subSchema = - calciteSchema.schema.getSubSchema(schemaName); - if (subSchema == null) { - throw new RuntimeException("sub-schema " + schemaName - + " not found"); - } - return new CachingCalciteSchema(calciteSchema, subSchema, - schemaName); - } - }); - } - } } diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java index 0217c05ad32..2c5a3d2ce12 100644 --- a/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java +++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java @@ -47,6 +47,7 @@ import org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractTableQueryable; import org.apache.calcite.schema.impl.MaterializedViewTable; +import org.apache.calcite.schema.lookup.LikePattern; import org.apache.calcite.server.CalciteServerStatement; import org.apache.calcite.sql.SqlJdbcFunctionCall; import org.apache.calcite.sql.SqlKind; @@ -79,7 +80,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.regex.Pattern; import static java.util.Objects.requireNonNull; @@ -227,52 +227,12 @@ public static CalciteMetaImpl create(CalciteConnection connection, } static Predicate1 namedMatcher(final Pat pattern) { - if (pattern.s == null || pattern.s.equals("%")) { - return Functions.truePredicate1(); - } - final Pattern regex = likeToRegex(pattern); - return v1 -> regex.matcher(v1.getName()).matches(); + final Predicate1 predicate = LikePattern.matcher(pattern.s); + return v1 -> predicate.apply(v1.getName()); } static Predicate1 matcher(final Pat pattern) { - if (pattern.s == null || pattern.s.equals("%")) { - return Functions.truePredicate1(); - } - final Pattern regex = likeToRegex(pattern); - return v1 -> regex.matcher(v1).matches(); - } - - /** Converts a LIKE-style pattern (where '%' represents a wild-card, escaped - * using '\') to a Java regex. */ - public static Pattern likeToRegex(Pat pattern) { - StringBuilder buf = new StringBuilder("^"); - char[] charArray = pattern.s.toCharArray(); - int slash = -2; - for (int i = 0; i < charArray.length; i++) { - char c = charArray[i]; - if (slash == i - 1) { - buf.append('[').append(c).append(']'); - } else { - switch (c) { - case '\\': - slash = i; - break; - case '%': - buf.append(".*"); - break; - case '[': - buf.append("\\["); - break; - case ']': - buf.append("\\]"); - break; - default: - buf.append('[').append(c).append(']'); - } - } - } - buf.append("$"); - return Pattern.compile(buf.toString()); + return LikePattern.matcher(pattern.s); } @Override public StatementHandle createStatement(ConnectionHandle ch) { @@ -397,7 +357,7 @@ private static ImmutableMap.Builder addProperty( final Predicate1 schemaMatcher = namedMatcher(schemaPattern); Enumerable tables = schemas(catalog) .where(schemaMatcher) - .selectMany(schema -> tables(schema, matcher(tableNamePattern))) + .selectMany(schema -> tables(schema, new LikePattern(tableNamePattern.s))) .where(typeFilter); return createResultSet(tables, metaTableFactory.getMetaTableClass(), @@ -414,13 +374,12 @@ private static ImmutableMap.Builder addProperty( Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) { - final Predicate1 tableNameMatcher = matcher(tableNamePattern); final Predicate1 schemaMatcher = namedMatcher(schemaPattern); final Predicate1 columnMatcher = namedMatcher(columnNamePattern); return createResultSet(schemas(catalog) .where(schemaMatcher) - .selectMany(schema -> tables(schema, tableNameMatcher)) + .selectMany(schema -> tables(schema, new LikePattern(tableNamePattern.s))) .selectMany(this::columns) .where(columnMatcher), metaColumnFactory.getMetaColumnClass(), @@ -458,12 +417,12 @@ Enumerable schemas(final String catalog) { Enumerable tables(String catalog) { return schemas(catalog) .selectMany(schema -> - tables(schema, Functions.truePredicate1())); + tables(schema, LikePattern.any())); } - Enumerable tables(final MetaSchema schema_) { + Enumerable tables(final MetaSchema schema_, LikePattern tableNamePattern) { final CalciteMetaSchema schema = (CalciteMetaSchema) schema_; - return Linq4j.asEnumerable(schema.calciteSchema.getTableNames()) + return Linq4j.asEnumerable(schema.calciteSchema.getTableNames(tableNamePattern)) .select(name -> { final Table table = requireNonNull(schema.calciteSchema.getTable(name, true), @@ -485,13 +444,6 @@ Enumerable tables(final MetaSchema schema_) { })); } - Enumerable tables( - final MetaSchema schema, - final Predicate1 matcher) { - return tables(schema) - .where(v1 -> matcher.apply(v1.getName())); - } - private ImmutableList getAllDefaultType() { final ImmutableList.Builder allTypeList = ImmutableList.builder(); diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java index 418b9e1a600..24de2d09820 100644 --- a/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java +++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java @@ -29,6 +29,10 @@ import org.apache.calcite.schema.Wrapper; import org.apache.calcite.schema.impl.MaterializedViewTable; import org.apache.calcite.schema.impl.StarTable; +import org.apache.calcite.schema.lookup.LikePattern; +import org.apache.calcite.schema.lookup.Lookup; +import org.apache.calcite.schema.lookup.Named; +import org.apache.calcite.util.LazyReference; import org.apache.calcite.util.NameMap; import org.apache.calcite.util.NameMultimap; import org.apache.calcite.util.NameSet; @@ -65,12 +69,14 @@ public abstract class CalciteSchema { /** Tables explicitly defined in this schema. Does not include tables in * {@link #schema}. */ protected final NameMap tableMap; + private final LazyReference> tables = new LazyReference<>(); protected final NameMultimap functionMap; protected final NameMap typeMap; protected final NameMap latticeMap; protected final NameSet functionNames; protected final NameMap nullaryFunctionMap; protected final NameMap subSchemaMap; + private final LazyReference> subSchemas = new LazyReference<>(); private @Nullable List> path; protected CalciteSchema(@Nullable CalciteSchema parent, Schema schema, @@ -109,17 +115,29 @@ protected CalciteSchema(@Nullable CalciteSchema parent, Schema schema, this.path = path; } - /** Returns a sub-schema with a given name that is defined implicitly - * (that is, by the underlying {@link Schema} object, not explicitly - * by a call to {@link #add(String, Schema)}), or null. */ - protected abstract @Nullable CalciteSchema getImplicitSubSchema(String schemaName, - boolean caseSensitive); + public Lookup tables() { + return this.tables.getOrCompute(() -> + Lookup.concat( + Lookup.of(this.tableMap), + decorateLookup(schema.tables().map((s, n) -> tableEntry(n, s))))); - /** Returns a table with a given name that is defined implicitly - * (that is, by the underlying {@link Schema} object, not explicitly - * by a call to {@link #add(String, Table)}), or null. */ - protected abstract @Nullable TableEntry getImplicitTable(String tableName, - boolean caseSensitive); + } + + public Lookup subSchemas() { + return subSchemas.getOrCompute(() -> + Lookup.concat( + Lookup.of(this.subSchemaMap), + decorateLookup(schema.subSchemas().map((s, n) -> createSubSchema(s, n))))); + } + + /** The derived class is able to decorate the lookup. */ + protected Lookup decorateLookup(Lookup lookup) { + return lookup; + } + + /** Creates a sub-schema with a given name that is defined implicitly. */ + protected abstract CalciteSchema createSubSchema(CalciteSchema this, + Schema schema, String name); /** Returns a type with a given name that is defined implicitly * (that is, by the underlying {@link Schema} object, not explicitly @@ -133,14 +151,6 @@ protected CalciteSchema(@Nullable CalciteSchema parent, Schema schema, protected abstract @Nullable TableEntry getImplicitTableBasedOnNullaryFunction(String tableName, boolean caseSensitive); - /** Adds implicit sub-schemas to a builder. */ - protected abstract void addImplicitSubSchemaToBuilder( - ImmutableSortedMap.Builder builder); - - /** Adds implicit tables to a builder. */ - protected abstract void addImplicitTableToBuilder( - ImmutableSortedSet.Builder builder); - /** Adds implicit functions to a builder. */ protected abstract void addImplicitFunctionsToBuilder( ImmutableList.Builder builder, @@ -250,13 +260,9 @@ public List path(@Nullable String name) { public final @Nullable CalciteSchema getSubSchema(String schemaName, boolean caseSensitive) { - // Check explicit schemas. - //noinspection LoopStatementThatDoesntLoop - for (Map.Entry entry - : subSchemaMap.range(schemaName, caseSensitive).entrySet()) { - return entry.getValue(); - } - return getImplicitSubSchema(schemaName, caseSensitive); + return caseSensitive + ? subSchemas().get(schemaName) + : Named.entityOrNull(subSchemas().getIgnoreCase(schemaName)); } /** Adds a child schema of this schema. */ @@ -274,13 +280,7 @@ public List path(@Nullable String name) { /** Returns a table with the given name. Does not look for views. */ public final @Nullable TableEntry getTable(String tableName, boolean caseSensitive) { - // Check explicit tables. - //noinspection LoopStatementThatDoesntLoop - for (Map.Entry entry - : tableMap.range(tableName, caseSensitive).entrySet()) { - return entry.getValue(); - } - return getImplicitTable(tableName, caseSensitive); + return Lookup.get(tables(), tableName, caseSensitive); } public String getName() { @@ -322,7 +322,6 @@ public final NavigableMap getSubSchemaMap() { final ImmutableSortedMap.Builder builder = new ImmutableSortedMap.Builder<>(NameSet.COMPARATOR); builder.putAll(subSchemaMap.map()); - addImplicitSubSchemaToBuilder(builder); return builder.build(); } @@ -335,14 +334,14 @@ public NavigableMap getLatticeMap() { /** Returns the set of all table names. Includes implicit and explicit tables * and functions with zero parameters. */ - public final NavigableSet getTableNames() { - final ImmutableSortedSet.Builder builder = - new ImmutableSortedSet.Builder<>(NameSet.COMPARATOR); - // Add explicit tables, case-sensitive. - builder.addAll(tableMap.map().keySet()); - // Add implicit tables, case-sensitive. - addImplicitTableToBuilder(builder); - return builder.build(); + public final Set getTableNames() { + return getTableNames(LikePattern.any()); + } + + /** Returns the set of filtered table names. Includes implicit and explicit tables + * and functions with zero parameters. */ + public final Set getTableNames(LikePattern pattern) { + return tables().getNames(pattern); } /** Returns the set of all types names. */ @@ -649,13 +648,21 @@ CalciteSchema calciteSchema() { return schema.getExpression(parentSchema, name); } + @Override public Lookup
tables() { + return CalciteSchema.this.tables().map((table, name) -> table.getTable()); + } + + @Override public Lookup subSchemas() { + return CalciteSchema.this.subSchemas().map((schema, name) -> schema.plus()); + } + @Override public @Nullable Table getTable(String name) { final TableEntry entry = CalciteSchema.this.getTable(name, true); return entry == null ? null : entry.getTable(); } - @Override public NavigableSet getTableNames() { - return CalciteSchema.this.getTableNames(); + @Override public Set getTableNames() { + return CalciteSchema.this.getTableNames(LikePattern.any()); } @Override public @Nullable RelProtoDataType getType(String name) { @@ -676,14 +683,11 @@ CalciteSchema calciteSchema() { } @Override public @Nullable SchemaPlus getSubSchema(String name) { - final CalciteSchema subSchema = - CalciteSchema.this.getSubSchema(name, true); - return subSchema == null ? null : subSchema.plus(); + return subSchemas().get(name); } @Override public Set getSubSchemaNames() { - //noinspection RedundantCast - return (Set) CalciteSchema.this.getSubSchemaMap().keySet(); + return subSchemas().getNames(LikePattern.any()); } @Override public SchemaPlus add(String name, Schema schema) { diff --git a/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java b/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java index edb1d2afde6..2d52edf6009 100644 --- a/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java +++ b/core/src/main/java/org/apache/calcite/jdbc/SimpleCalciteSchema.java @@ -100,36 +100,8 @@ private SimpleCalciteSchema(@Nullable CalciteSchema parent, return null; } - @Override protected @Nullable CalciteSchema getImplicitSubSchema(String schemaName, - boolean caseSensitive) { - // Check implicit schemas. - final String schemaName2 = - caseSensitive ? schemaName - : caseInsensitiveLookup(schema.getSubSchemaNames(), schemaName); - if (schemaName2 == null) { - return null; - } - final Schema s = schema.getSubSchema(schemaName2); - if (s == null) { - return null; - } - return new SimpleCalciteSchema(this, s, schemaName2); - } - - @Override protected @Nullable TableEntry getImplicitTable(String tableName, - boolean caseSensitive) { - // Check implicit tables. - final String tableName2 = - caseSensitive ? tableName - : caseInsensitiveLookup(schema.getTableNames(), tableName); - if (tableName2 == null) { - return null; - } - final Table table = schema.getTable(tableName2); - if (table == null) { - return null; - } - return tableEntry(tableName2, table); + @Override protected CalciteSchema createSubSchema(Schema schema, String name) { + return new SimpleCalciteSchema(this, schema, name); } @Override protected @Nullable TypeEntry getImplicitType(String name, boolean caseSensitive) { @@ -147,26 +119,6 @@ private SimpleCalciteSchema(@Nullable CalciteSchema parent, return typeEntry(name2, type); } - @Override protected void addImplicitSubSchemaToBuilder( - ImmutableSortedMap.Builder builder) { - ImmutableSortedMap explicitSubSchemas = builder.build(); - for (String schemaName : schema.getSubSchemaNames()) { - if (explicitSubSchemas.containsKey(schemaName)) { - // explicit subschema wins. - continue; - } - Schema s = schema.getSubSchema(schemaName); - if (s != null) { - CalciteSchema calciteSchema = new SimpleCalciteSchema(this, s, schemaName); - builder.put(schemaName, calciteSchema); - } - } - } - - @Override protected void addImplicitTableToBuilder(ImmutableSortedSet.Builder builder) { - builder.addAll(schema.getTableNames()); - } - @Override protected void addImplicitFunctionsToBuilder( ImmutableList.Builder builder, String name, boolean caseSensitive) { diff --git a/core/src/main/java/org/apache/calcite/model/ModelHandler.java b/core/src/main/java/org/apache/calcite/model/ModelHandler.java index cd99332f752..65f9fd65977 100644 --- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java +++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java @@ -40,6 +40,7 @@ import org.apache.calcite.schema.impl.TableFunctionImpl; import org.apache.calcite.schema.impl.TableMacroImpl; import org.apache.calcite.schema.impl.ViewTable; +import org.apache.calcite.schema.lookup.LikePattern; import org.apache.calcite.sql.SqlDialectFactory; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.Pair; @@ -361,7 +362,7 @@ public void visit(JsonMaterialization jsonMaterialization) { if (jsonMaterialization.view == null) { // If the user did not supply a view name, that means the materialized // view is pre-populated. Generate a synthetic view name. - viewName = "$" + schema.getTableNames().size(); + viewName = "$" + schema.tables().getNames(LikePattern.any()).size(); existing = true; } else { viewName = jsonMaterialization.view; diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java index 264d6e88c83..cec43afca67 100644 --- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java +++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java @@ -33,6 +33,7 @@ import org.apache.calcite.schema.TableMacro; import org.apache.calcite.schema.Wrapper; import org.apache.calcite.schema.impl.ScalarFunctionImpl; +import org.apache.calcite.schema.lookup.LikePattern; import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; @@ -205,7 +206,7 @@ private Collection getFunctionsFrom( result.add(moniker(schema, subSchema, SqlMonikerType.SCHEMA)); } - for (String table : schema.getTableNames()) { + for (String table : schema.getTableNames(LikePattern.any())) { result.add(moniker(schema, table, SqlMonikerType.TABLE)); } diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java index 29a6f07c014..e9883a3f121 100644 --- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java +++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java @@ -47,12 +47,15 @@ import org.apache.calcite.schema.TemporalTable; import org.apache.calcite.schema.TranslatableTable; import org.apache.calcite.schema.Wrapper; +import org.apache.calcite.schema.lookup.LikePattern; +import org.apache.calcite.schema.lookup.Lookup; import org.apache.calcite.sql.SqlAccessType; import org.apache.calcite.sql.validate.SqlModality; import org.apache.calcite.sql.validate.SqlMonotonicity; import org.apache.calcite.sql2rel.InitializerExpressionFactory; import org.apache.calcite.sql2rel.NullInitializerExpressionFactory; import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.LazyReference; import org.apache.calcite.util.Pair; import org.apache.calcite.util.Util; @@ -429,6 +432,8 @@ private static class MySchemaPlus implements SchemaPlus { private final @Nullable SchemaPlus parent; private final String name; private final Schema schema; + private final LazyReference> subSchemas = new LazyReference<>(); + MySchemaPlus(@Nullable SchemaPlus parent, String name, Schema schema) { this.parent = parent; @@ -456,8 +461,7 @@ public static MySchemaPlus create(Path path) { } @Override public @Nullable SchemaPlus getSubSchema(String name) { - final Schema subSchema = schema.getSubSchema(name); - return subSchema == null ? null : new MySchemaPlus(this, name, subSchema); + return subSchemas().get(name); } @Override public SchemaPlus add(String name, Schema schema) { @@ -505,12 +509,21 @@ public static MySchemaPlus create(Path path) { return false; } + @Override public Lookup
tables() { + return schema.tables(); + } + + @Override public Lookup subSchemas() { + return subSchemas.getOrCompute( + () -> schema.subSchemas().map((s, key) -> new MySchemaPlus(this, key, s))); + } + @Override public @Nullable Table getTable(String name) { - return schema.getTable(name); + return tables().get(name); } @Override public Set getTableNames() { - return schema.getTableNames(); + return schema.tables().getNames(LikePattern.any()); } @Override public @Nullable RelProtoDataType getType(String name) { @@ -531,7 +544,7 @@ public static MySchemaPlus create(Path path) { } @Override public Set getSubSchemaNames() { - return schema.getSubSchemaNames(); + return schema.subSchemas().getNames(LikePattern.any()); } @Override public Expression getExpression(@Nullable SchemaPlus parentSchema, diff --git a/core/src/main/java/org/apache/calcite/schema/Schema.java b/core/src/main/java/org/apache/calcite/schema/Schema.java index 5e034b1da9d..bd0dcd32616 100644 --- a/core/src/main/java/org/apache/calcite/schema/Schema.java +++ b/core/src/main/java/org/apache/calcite/schema/Schema.java @@ -18,6 +18,9 @@ import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.schema.lookup.CompatibilityLookup; +import org.apache.calcite.schema.lookup.LikePattern; +import org.apache.calcite.schema.lookup.Lookup; import org.checkerframework.checker.nullness.qual.Nullable; @@ -56,9 +59,30 @@ * {@link Schema#getSubSchema(String)}. */ public interface Schema { + + /** + * Returns a lookup object to find tables. + * + * @return Lookup + */ + default Lookup
tables() { + return new CompatibilityLookup<>(this::getTable, this::getTableNames); + } + + /** + * Returns a lookup object to find sub schemas. + * + * @return Lookup + */ + default Lookup subSchemas() { + return new CompatibilityLookup<>(this::getSubSchema, this::getSubSchemaNames); + } + /** * Returns a table with a given name, or null if not found. * + *

Please use {@link Schema#tables()} and {@link Lookup#get(String)} instead. + * * @param name Table name * @return Table, or null */ @@ -67,6 +91,8 @@ public interface Schema { /** * Returns the names of the tables in this schema. * + *

Please use {@link Schema#tables()} and {@link Lookup#getNames(LikePattern)} instead. + * * @return Names of the tables in this schema */ Set getTableNames(); @@ -105,6 +131,8 @@ public interface Schema { /** * Returns a sub-schema with a given name, or null. * + *

Please use {@link Schema#subSchemas()} and {@link Lookup#get(String)} instead. + * * @param name Sub-schema name * @return Sub-schema with a given name, or null */ @@ -113,6 +141,8 @@ public interface Schema { /** * Returns the names of this schema's child schemas. * + *

Please use {@link Schema#subSchemas()} and {@link Lookup#getNames(LikePattern)} instead. + * * @return Names of this schema's child schemas */ Set getSubSchemaNames(); diff --git a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java index 5f417886543..c941bb051f9 100644 --- a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java +++ b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java @@ -18,6 +18,7 @@ import org.apache.calcite.materialize.Lattice; import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.schema.lookup.Lookup; import com.google.common.collect.ImmutableList; @@ -45,6 +46,11 @@ * {@link Schema}, or indeed might not. */ public interface SchemaPlus extends Schema { + + /** + * Returns a lookup object to find sub schemas. + */ + @Override Lookup subSchemas(); /** * Returns the parent schema, or null if this schema has no parent. */ diff --git a/core/src/main/java/org/apache/calcite/schema/Schemas.java b/core/src/main/java/org/apache/calcite/schema/Schemas.java index 1fafb231237..8abff8dd8af 100644 --- a/core/src/main/java/org/apache/calcite/schema/Schemas.java +++ b/core/src/main/java/org/apache/calcite/schema/Schemas.java @@ -235,7 +235,7 @@ public static Queryable queryable(DataContext root, Class clazz, String name = iterator.next(); requireNonNull(schema, "schema"); if (iterator.hasNext()) { - SchemaPlus next = schema.getSubSchema(name); + SchemaPlus next = schema.subSchemas().get(name); if (next == null) { throw new IllegalArgumentException("schema " + name + " is not found in " + schema); } @@ -250,7 +250,7 @@ public static Queryable queryable(DataContext root, Class clazz, public static Queryable queryable(DataContext root, SchemaPlus schema, Class clazz, String tableName) { QueryableTable table = - (QueryableTable) requireNonNull(schema.getTable(tableName), + (QueryableTable) requireNonNull(schema.tables().get(tableName), () -> "table " + tableName + " is not found in " + schema); QueryProvider queryProvider = root.getQueryProvider(); return table.asQueryable(queryProvider, schema, tableName); @@ -299,13 +299,13 @@ private static int[] identity(int count) { String name = iterator.next(); requireNonNull(schema, "schema"); if (iterator.hasNext()) { - SchemaPlus next = schema.getSubSchema(name); + SchemaPlus next = schema.subSchemas().get(name); if (next == null) { throw new IllegalArgumentException("schema " + name + " is not found in " + schema); } schema = next; } else { - return schema.getTable(name); + return schema.tables().get(name); } } } @@ -576,7 +576,7 @@ public static Path path(CalciteSchema rootSchema, Iterable names) { if (!iterator.hasNext()) { return path(builder.build()); } - Schema next = schema.getSubSchema(name); + Schema next = schema.subSchemas().get(name); if (next == null) { throw new IllegalArgumentException("schema " + name + " is not found in " + schema); } diff --git a/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java b/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java index a7b050a5045..6a27b621b6a 100644 --- a/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java +++ b/core/src/main/java/org/apache/calcite/schema/impl/AbstractSchema.java @@ -25,6 +25,9 @@ import org.apache.calcite.schema.SchemaVersion; import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; +import org.apache.calcite.schema.lookup.CompatibilityLookup; +import org.apache.calcite.schema.lookup.Lookup; +import org.apache.calcite.util.LazyReference; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultimap; @@ -56,7 +59,18 @@ * */ public class AbstractSchema implements Schema { - public AbstractSchema() { + + private LazyReference> tables = new LazyReference<>(); + private LazyReference> subSchemas = new LazyReference<>(); + + @Override public Lookup

tables() { + return tables.getOrCompute( + () -> new CompatibilityLookup<>(this::getTable, this::getTableNames)); + } + + @Override public Lookup subSchemas() { + return subSchemas.getOrCompute( + () -> new CompatibilityLookup<>(this::getSubSchema, this::getSubSchemaNames)); } @Override public boolean isMutable() { diff --git a/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java b/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java index e63f518878b..316b50d804b 100644 --- a/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java +++ b/core/src/main/java/org/apache/calcite/schema/impl/DelegatingSchema.java @@ -23,6 +23,8 @@ import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.SchemaVersion; import org.apache.calcite.schema.Table; +import org.apache.calcite.schema.lookup.LikePattern; +import org.apache.calcite.schema.lookup.Lookup; import org.checkerframework.checker.nullness.qual.Nullable; @@ -61,12 +63,16 @@ public DelegatingSchema(Schema schema) { return schema.getExpression(parentSchema, name); } + @Override public Lookup
tables() { + return schema.tables(); + } + @Override public @Nullable Table getTable(String name) { - return schema.getTable(name); + return schema.tables().get(name); } @Override public Set getTableNames() { - return schema.getTableNames(); + return schema.tables().getNames(LikePattern.any()); } @Override public @Nullable RelProtoDataType getType(String name) { @@ -85,11 +91,15 @@ public DelegatingSchema(Schema schema) { return schema.getFunctionNames(); } + @Override public Lookup subSchemas() { + return schema.subSchemas(); + } + @Override public @Nullable Schema getSubSchema(String name) { - return schema.getSubSchema(name); + return subSchemas().get(name); } @Override public Set getSubSchemaNames() { - return schema.getSubSchemaNames(); + return subSchemas().getNames(LikePattern.any()); } } diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/CachedLookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/CachedLookup.java new file mode 100644 index 00000000000..82fc097cd1a --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/CachedLookup.java @@ -0,0 +1,78 @@ +/* + * 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.calcite.schema.lookup; + +import org.apache.calcite.util.LazyReference; +import org.apache.calcite.util.NameMap; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Set; + +/** + * This class can be used to make a snapshot of a lookups. + * + * @param Element Type + */ +public class CachedLookup implements Lookup { + + private final Lookup delegate; + private LazyReference> cachedDelegate = new LazyReference<>(); + private boolean enabled = true; + + public CachedLookup(Lookup delegate) { + this.delegate = delegate; + } + + @Override public @Nullable T get(final String name) { + return delegate().get(name); + } + + @Override public @Nullable Named getIgnoreCase(final String name) { + return delegate().getIgnoreCase(name); + } + + @Override public Set getNames(final LikePattern pattern) { + return delegate().getNames(pattern); + } + + private Lookup delegate() { + if (!enabled) { + return delegate; + } + return cachedDelegate.getOrCompute(() -> new NameMapLookup<>(loadNameMap())); + } + + private NameMap loadNameMap() { + NameMap map = new NameMap<>(); + for (String name : delegate.getNames(LikePattern.any())) { + T entry = delegate.get(name); + if (entry != null) { + map.put(name, entry); + } + } + return map; + } + + public void enable(boolean enabled) { + if (!enabled) { + cachedDelegate.reset(); + } + this.enabled = enabled; + } + +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/CompatibilityLookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/CompatibilityLookup.java new file mode 100644 index 00000000000..5f34129f2ce --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/CompatibilityLookup.java @@ -0,0 +1,54 @@ +/* + * 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.calcite.schema.lookup; + +import org.apache.calcite.linq4j.function.Predicate1; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** + * This class can be used to wrap existing schemas with a pair of {@code get...} + * and {@code get...Names} into a Lookup object. + * + * @param Element type + */ +public class CompatibilityLookup extends IgnoreCaseLookup { + + private final Function get; + private final Supplier> getNames; + + public CompatibilityLookup(Function get, Supplier> getNames) { + this.get = get; + this.getNames = getNames; + } + + @Override public @Nullable T get(String name) { + return get.apply(name); + } + + @Override public Set getNames(LikePattern pattern) { + final Predicate1 matcher = pattern.matcher(); + return getNames.get().stream() + .filter(name -> matcher.apply(name)) + .collect(Collectors.toSet()); + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/ConcatLookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/ConcatLookup.java new file mode 100644 index 00000000000..149b1985e9c --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/ConcatLookup.java @@ -0,0 +1,62 @@ +/* + * 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.calcite.schema.lookup; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * This class can be used to concat a list of lookups. + * + * @param Element type + */ +class ConcatLookup implements Lookup { + private final Lookup[] lookups; + + ConcatLookup(Lookup[] lookups) { + this.lookups = lookups; + } + + @Override public @Nullable T get(String name) { + for (Lookup lookup : lookups) { + T t = lookup.get(name); + if (t != null) { + return t; + } + } + return null; + } + + @Override public @Nullable Named getIgnoreCase(String name) { + for (Lookup lookup : lookups) { + Named t = lookup.getIgnoreCase(name); + if (t != null) { + return t; + } + } + return null; + } + + @Override public Set getNames(LikePattern pattern) { + return Stream.of(lookups) + .flatMap(lookup -> lookup.getNames(pattern).stream()) + .collect(Collectors.toSet()); + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/EmptyLookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/EmptyLookup.java new file mode 100644 index 00000000000..7488b24cab4 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/EmptyLookup.java @@ -0,0 +1,45 @@ +/* + * 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.calcite.schema.lookup; + +import com.google.common.collect.ImmutableSet; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Set; + +/** + * This class implements an empty Lookup. + * + * @param Element type + */ +class EmptyLookup implements Lookup { + + static final Lookup INSTANCE = new EmptyLookup<>(); + + @Override public @Nullable T get(String name) { + return null; + } + + @Override public @Nullable Named getIgnoreCase(String name) { + return null; + } + + @Override public Set getNames(LikePattern pattern) { + return ImmutableSet.of(); + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/IgnoreCaseLookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/IgnoreCaseLookup.java new file mode 100644 index 00000000000..0590315c4c9 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/IgnoreCaseLookup.java @@ -0,0 +1,77 @@ +/* + * 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.calcite.schema.lookup; + +import org.apache.calcite.util.LazyReference; +import org.apache.calcite.util.NameMap; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Map; +import java.util.Set; + +/** + * An abstract base class for lookups. implementing case insensitive lookup + * + * @param Element type + */ +public abstract class IgnoreCaseLookup implements Lookup { + + private LazyReference> nameMap = new LazyReference<>(); + + /** + * Returns a named entity with a given name, or null if not found. + * + * @param name Name + * @return Entity, or null + */ + @Override public abstract @Nullable T get(String name); + + /** + * Returns a named entity with a given name ignoring the case, or null if not found. + * + * @param name Name + * @return Entity, or null + */ + @Override @Nullable public Named getIgnoreCase(String name) { + int retryCounter = 0; + while (true) { + Map.Entry entry = nameMap.getOrCompute(this::loadNames) + .range(name, false) + .firstEntry(); + if (entry != null) { + T result = get(entry.getValue()); + return result == null ? null : new Named<>(entry.getKey(), result); + } + retryCounter++; + if (retryCounter > 1) { + return null; + } + nameMap.reset(); + } + } + + @Override public abstract Set getNames(LikePattern pattern); + + private NameMap loadNames() { + NameMap result = new NameMap<>(); + for (String name : getNames(LikePattern.any())) { + result.put(name, name); + } + return result; + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/LikePattern.java b/core/src/main/java/org/apache/calcite/schema/lookup/LikePattern.java new file mode 100644 index 00000000000..79603c1e501 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/LikePattern.java @@ -0,0 +1,92 @@ +/* + * 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.calcite.schema.lookup; + +import org.apache.calcite.linq4j.function.Predicate1; + +import java.util.regex.Pattern; + +/** + * This class is used as parameter to Lookup.getNames + */ +public class LikePattern { + private static final String ANY = "%"; + public final String pattern; + + public LikePattern(String pattern) { + if (pattern == null) { + pattern = ANY; + } + this.pattern = pattern; + } + + @Override public String toString() { + return "LikePattern[" + this.pattern + "]"; + } + + public Predicate1 matcher() { + return matcher(pattern); + } + + public static LikePattern any() { + return new LikePattern(ANY); + } + + public static Predicate1 matcher(String likePattern) { + if (likePattern == null || likePattern.equals(ANY)) { + return v1 -> true; + } + final Pattern regex = likeToRegex(likePattern); + return v1 -> regex.matcher(v1).matches(); + } + + /** + * Converts a LIKE-style pattern (where '%' represents a wild-card, escaped + * using '\') to a Java regex. + */ + public static Pattern likeToRegex(String pattern) { + StringBuilder buf = new StringBuilder("^"); + char[] charArray = pattern.toCharArray(); + int slash = -2; + for (int i = 0; i < charArray.length; i++) { + char c = charArray[i]; + if (slash == i - 1) { + buf.append('[').append(c).append(']'); + } else { + switch (c) { + case '\\': + slash = i; + break; + case '%': + buf.append(".*"); + break; + case '[': + buf.append("\\["); + break; + case ']': + buf.append("\\]"); + break; + default: + buf.append('[').append(c).append(']'); + } + } + } + buf.append("$"); + return Pattern.compile(buf.toString()); + } + +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/LoadingCacheLookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/LoadingCacheLookup.java new file mode 100644 index 00000000000..139a72d52b1 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/LoadingCacheLookup.java @@ -0,0 +1,82 @@ +/* + * 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.calcite.schema.lookup; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.util.concurrent.UncheckedExecutionException; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static java.util.Objects.requireNonNull; + +/** + * This class can be used to cache lookups. + * + * @param Element Type + */ +public class LoadingCacheLookup implements Lookup { + private final Lookup delegate; + + private final LoadingCache cache; + private final LoadingCache> cacheIgnoreCase; + + public LoadingCacheLookup(Lookup delegate) { + this.delegate = delegate; + this.cache = CacheBuilder.newBuilder() + .expireAfterWrite(1, TimeUnit.MINUTES) + .build(CacheLoader.from(name -> requireNonNull(delegate.get(name)))); + this.cacheIgnoreCase = CacheBuilder.newBuilder() + .expireAfterWrite(1, TimeUnit.MINUTES) + .build(CacheLoader.from(name -> requireNonNull(delegate.getIgnoreCase(name)))); + } + + @Override public @Nullable T get(String name) { + try { + return cache.get(name); + } catch (UncheckedExecutionException e) { + if (e.getCause() instanceof NullPointerException) { + return null; + } + throw e; + } catch (ExecutionException e) { + throw new RuntimeException(e); + } + } + + @Override public @Nullable Named getIgnoreCase(String name) { + try { + return cacheIgnoreCase.get(name); + } catch (UncheckedExecutionException e) { + if (e.getCause() instanceof NullPointerException) { + return null; + } + throw e; + } catch (ExecutionException e) { + throw new RuntimeException(e); + } + } + + @Override public Set getNames(LikePattern pattern) { + return delegate.getNames(pattern); + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/Lookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/Lookup.java new file mode 100644 index 00000000000..446a4961d4d --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/Lookup.java @@ -0,0 +1,96 @@ +/* + * 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.calcite.schema.lookup; + +import org.apache.calcite.util.NameMap; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Set; +import java.util.function.BiFunction; + +/** + * A casesensitive/insensitive lookup for tables, schems, functions ... + * + * @param Element type + */ +public interface Lookup { + /** + * Returns a named entity with a given name, or null if not found. + * + * @param name Name + * @return Entity, or null + */ + @Nullable T get(String name); + + /** + * Returns a named entity with a given name ignoring the case, or null if not found. + * + * @param name Name + * @return Entity, or null + */ + @Nullable Named getIgnoreCase(String name); + + /** + * Returns the names of the entities in matching pattern. + * + * @return Names of the entities + */ + Set getNames(LikePattern pattern); + + default Lookup map(BiFunction mapper) { + return new MappedLookup<>(this, mapper); + } + + /** + * Helper method to call Lookup.get(String) or Lookup.getIgnoreCase(String) + * depending on the parameter caseSensitive. + * + * @return Entity, or null + */ + static @Nullable T get(Lookup lookup, String name, boolean caseSensitive) { + if (caseSensitive) { + T entry = lookup.get(name); + if (entry == null) { + return null; + } + return entry; + } + return Named.entityOrNull(lookup.getIgnoreCase(name)); + } + + /** + * Returns an empty lookup. + */ + static Lookup empty() { + return (Lookup) EmptyLookup.INSTANCE; + } + + /** + * Creates a new lookup object based on a NameMap. + */ + static Lookup of(NameMap map) { + return new NameMapLookup<>(map); + } + + /** + * Concat a list of lookups. + */ + static Lookup concat(Lookup... lookups) { + return new ConcatLookup<>(lookups); + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/MappedLookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/MappedLookup.java new file mode 100644 index 00000000000..fae372b324e --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/MappedLookup.java @@ -0,0 +1,54 @@ +/* + * 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.calcite.schema.lookup; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Set; +import java.util.function.BiFunction; + +/** + * A Lookup class which can be used to map different element types. + * + * @param Source element type + * @param Target element type + */ +class MappedLookup implements Lookup { + private final Lookup lookup; + private final BiFunction mapper; + + MappedLookup(Lookup lookup, BiFunction mapper) { + this.lookup = lookup; + this.mapper = mapper; + } + + @Override public @Nullable T get(String name) { + S entity = lookup.get(name); + return entity == null ? null : mapper.apply(entity, name); + } + + @Override public @Nullable Named getIgnoreCase(String name) { + Named named = lookup.getIgnoreCase(name); + return named == null + ? null + : new Named<>(named.name(), mapper.apply(named.entity(), named.name())); + } + + @Override public Set getNames(LikePattern pattern) { + return lookup.getNames(pattern); + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/NameMapLookup.java b/core/src/main/java/org/apache/calcite/schema/lookup/NameMapLookup.java new file mode 100644 index 00000000000..7231eea6696 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/NameMapLookup.java @@ -0,0 +1,62 @@ +/* + * 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.calcite.schema.lookup; + +import org.apache.calcite.linq4j.function.Predicate1; +import org.apache.calcite.util.NameMap; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A Lookup class which is based on a NameMap. + * + * @param Element type + */ +class NameMapLookup implements Lookup { + private final NameMap map; + + NameMapLookup(NameMap map) { + this.map = map; + } + + @Override public @Nullable T get(String name) { + Map.Entry entry = map.range(name, true).firstEntry(); + if (entry != null) { + return entry.getValue(); + } + return null; + } + + @Override public @Nullable Named getIgnoreCase(String name) { + Map.Entry entry = map.range(name, false).firstEntry(); + if (entry != null) { + return new Named<>(entry.getKey(), entry.getValue()); + } + return null; + } + + @Override public Set getNames(LikePattern pattern) { + final Predicate1 matcher = pattern.matcher(); + return map.map().keySet().stream() + .filter(name -> matcher.apply(name)) + .collect(Collectors.toSet()); + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/Named.java b/core/src/main/java/org/apache/calcite/schema/lookup/Named.java new file mode 100644 index 00000000000..ec9c672f2fa --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/Named.java @@ -0,0 +1,67 @@ +/* + * 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.calcite.schema.lookup; + +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; + +import static java.util.Objects.requireNonNull; + +/** + * This class is used to hold an object including its name. + * + * @param Element type + */ +public class Named { + private final String name; + private final @NonNull T entity; + + public Named(String name, T entity) { + this.name = name; + this.entity = requireNonNull(entity, "entity"); + } + + public final String name() { + return name; + } + + public final T entity() { + return entity; + } + + public static @Nullable T entityOrNull(@Nullable Named named) { + return named == null ? null : named.entity; + } + + @Override public boolean equals(final @Nullable Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + final Named named = (Named) o; + return name.equals(named.name) && entity.equals(named.entity); + } + + @Override public int hashCode() { + int result = name.hashCode(); + result = 31 * result + entity.hashCode(); + return result; + } +} diff --git a/core/src/main/java/org/apache/calcite/schema/lookup/package-info.java b/core/src/main/java/org/apache/calcite/schema/lookup/package-info.java new file mode 100644 index 00000000000..edd5b38d382 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/lookup/package-info.java @@ -0,0 +1,31 @@ +/* + * 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. + */ + +/** + * Schema Lookup SPI. + * + *

The interfaces and classes in this package are used to lookup + * tables and subschemas within a schema. + */ +@DefaultQualifier(value = NonNull.class, locations = TypeUseLocation.FIELD) +@DefaultQualifier(value = NonNull.class, locations = TypeUseLocation.PARAMETER) +@DefaultQualifier(value = NonNull.class, locations = TypeUseLocation.RETURN) +package org.apache.calcite.schema.lookup; + +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.framework.qual.DefaultQualifier; +import org.checkerframework.framework.qual.TypeUseLocation; diff --git a/core/src/main/java/org/apache/calcite/util/LazyReference.java b/core/src/main/java/org/apache/calcite/util/LazyReference.java new file mode 100644 index 00000000000..efed04e2eed --- /dev/null +++ b/core/src/main/java/org/apache/calcite/util/LazyReference.java @@ -0,0 +1,47 @@ +/* + * 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.calcite.util; + +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; + +/** + * This class can be used to lazily initialize an object. + * + * @param Element Type + */ +public class LazyReference { + + private final AtomicReference value = new AtomicReference<>(); + + public T getOrCompute(Supplier supplier) { + while (true) { + T result = value.get(); + if (result != null) { + return result; + } + T computed = supplier.get(); + if (value.compareAndSet((T) null, computed)) { + return computed; + } + } + } + + public void reset() { + value.set((T) null); + } +} diff --git a/core/src/test/java/org/apache/calcite/schema/lookup/CachedLookupTest.java b/core/src/test/java/org/apache/calcite/schema/lookup/CachedLookupTest.java new file mode 100644 index 00000000000..ec7f8d3f5e0 --- /dev/null +++ b/core/src/test/java/org/apache/calcite/schema/lookup/CachedLookupTest.java @@ -0,0 +1,43 @@ +/* + * 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.calcite.schema.lookup; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +/** + * Test for CachedLookup. + */ +class CachedLookupTest { + private final Lookup testee = new CachedLookup<>(new MapLookup("a", "1")); + + @Test void testNull() { + assertThat(testee.get("c"), nullValue()); + } + + @Test void test() { + assertThat(testee.get("a"), equalTo("1")); + } + + @Test void testIgnoreCase() { + assertThat(testee.getIgnoreCase("A"), equalTo(new Named<>("a", "1"))); + } + +} diff --git a/core/src/test/java/org/apache/calcite/schema/lookup/ConcatLookupTest.java b/core/src/test/java/org/apache/calcite/schema/lookup/ConcatLookupTest.java new file mode 100644 index 00000000000..e84ea3f8b3f --- /dev/null +++ b/core/src/test/java/org/apache/calcite/schema/lookup/ConcatLookupTest.java @@ -0,0 +1,43 @@ +/* + * 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.calcite.schema.lookup; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +/** + * Test for ConcatLookup. + */ +class ConcatLookupTest { + private final Lookup testee = + Lookup.concat(new MapLookup("a", "1"), new MapLookup("b", "2")); + + @Test void testNull() { + assertThat(testee.get("c"), nullValue()); + } + + @Test void test() { + assertThat(testee.get("a"), equalTo("1")); + } + + @Test void testIgnoreCase() { + assertThat(testee.getIgnoreCase("B"), equalTo(new Named<>("b", "2"))); + } +} diff --git a/core/src/test/java/org/apache/calcite/schema/lookup/IgnoreCaseLookupTest.java b/core/src/test/java/org/apache/calcite/schema/lookup/IgnoreCaseLookupTest.java new file mode 100644 index 00000000000..95aef847388 --- /dev/null +++ b/core/src/test/java/org/apache/calcite/schema/lookup/IgnoreCaseLookupTest.java @@ -0,0 +1,58 @@ +/* + * 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.calcite.schema.lookup; + +import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Set; + +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +/** + * Test for IgnoreCaseLookup. + */ +class IgnoreCaseLookupTest { + private final Lookup testee = new IgnoreCaseLookup() { + @Override public @Nullable String get(final String name) { + if ("a".equals(name)) { + return "1"; + } + return null; + } + + @Override public Set getNames(final LikePattern pattern) { + return Collections.singleton("a"); + } + }; + + @Test void testNull() { + assertThat(testee.get("c"), nullValue()); + } + + @Test void test() { + assertThat(testee.get("a"), equalTo("1")); + } + + @Test void testIgnoreCase() { + assertThat(testee.getIgnoreCase("A"), equalTo(new Named<>("a", "1"))); + } + +} diff --git a/core/src/test/java/org/apache/calcite/schema/lookup/LoadingCacheLookupTest.java b/core/src/test/java/org/apache/calcite/schema/lookup/LoadingCacheLookupTest.java new file mode 100644 index 00000000000..0d5e6f1e146 --- /dev/null +++ b/core/src/test/java/org/apache/calcite/schema/lookup/LoadingCacheLookupTest.java @@ -0,0 +1,45 @@ +/* + * 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.calcite.schema.lookup; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +/** + * Test for LoadingCacheLookup. + */ +public class LoadingCacheLookupTest { + + private final Lookup testee = + new LoadingCacheLookup<>(new MapLookup("test", "xxxx")); + + @Test void testNull() { + assertThat(testee.get("unknown"), nullValue()); + } + + @Test void test() { + assertThat(testee.get("test"), equalTo("xxxx")); + } + + @Test void testIgnoreCase() { + assertThat(testee.getIgnoreCase("TEST"), equalTo(new Named<>("test", "xxxx"))); + } + +} diff --git a/core/src/test/java/org/apache/calcite/schema/lookup/MapLookup.java b/core/src/test/java/org/apache/calcite/schema/lookup/MapLookup.java new file mode 100644 index 00000000000..5150c429a8b --- /dev/null +++ b/core/src/test/java/org/apache/calcite/schema/lookup/MapLookup.java @@ -0,0 +1,57 @@ +/* + * 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.calcite.schema.lookup; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Simple test class for Lookup. + */ +class MapLookup implements Lookup { + private final Map map; + private final Map> ignoreCaseMap; + + MapLookup(String... keyAndValues) { + this.map = new HashMap<>(); + for (int i = 0; i < keyAndValues.length - 1; i += 2) { + map.put(keyAndValues[i], keyAndValues[i + 1]); + } + this.ignoreCaseMap = this.map.entrySet().stream() + .collect( + Collectors.toMap( + entry -> entry.getKey().toLowerCase(Locale.ROOT), + entry -> new Named<>(entry.getKey(), entry.getValue()))); + } + + @Override public @Nullable String get(final String name) { + return map.get(name); + } + + @Override public @Nullable Named getIgnoreCase(final String name) { + return ignoreCaseMap.get(name.toLowerCase(Locale.ROOT)); + } + + @Override public Set getNames(final LikePattern pattern) { + return map.keySet(); + } +} diff --git a/core/src/test/java/org/apache/calcite/schema/lookup/MappedLookupTest.java b/core/src/test/java/org/apache/calcite/schema/lookup/MappedLookupTest.java new file mode 100644 index 00000000000..f68407d0c35 --- /dev/null +++ b/core/src/test/java/org/apache/calcite/schema/lookup/MappedLookupTest.java @@ -0,0 +1,44 @@ +/* + * 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.calcite.schema.lookup; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +/** + * Test for MappedLookup. + */ +class MappedLookupTest { + private final Lookup testee = + (new MapLookup("a", "1")).map((value, name) -> name + "_" + value); + + @Test void testNull() { + assertThat(testee.get("c"), nullValue()); + } + + @Test void test() { + assertThat(testee.get("a"), equalTo("a_1")); + } + + @Test void testIgnoreCase() { + assertThat(testee.getIgnoreCase("A"), equalTo(new Named<>("a", "a_1"))); + } + +} diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java index f08e99322e8..5919cb76f17 100644 --- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java +++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java @@ -29,7 +29,6 @@ import org.apache.calcite.avatica.AvaticaStatement; import org.apache.calcite.avatica.Handler; import org.apache.calcite.avatica.HandlerImpl; -import org.apache.calcite.avatica.Meta; import org.apache.calcite.avatica.util.Casing; import org.apache.calcite.avatica.util.Quoting; import org.apache.calcite.config.CalciteConnectionConfig; @@ -38,7 +37,6 @@ import org.apache.calcite.config.Lex; import org.apache.calcite.config.NullCollation; import org.apache.calcite.jdbc.CalciteConnection; -import org.apache.calcite.jdbc.CalciteMetaImpl; import org.apache.calcite.jdbc.CalcitePrepare; import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.jdbc.Driver; @@ -74,6 +72,7 @@ import org.apache.calcite.schema.impl.DelegatingSchema; import org.apache.calcite.schema.impl.TableMacroImpl; import org.apache.calcite.schema.impl.ViewTable; +import org.apache.calcite.schema.lookup.LikePattern; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlDialect; import org.apache.calcite.sql.SqlKind; @@ -1078,7 +1077,7 @@ private String mm(int majorVersion, int minorVersion) { } /** Unit test for - * {@link org.apache.calcite.jdbc.CalciteMetaImpl#likeToRegex(org.apache.calcite.avatica.Meta.Pat)}. */ + * {@link LikePattern#likeToRegex(org.apache.calcite.avatica.Meta.Pat)}. */ @Test void testLikeToRegex() { checkLikeToRegex(true, "%", "abc"); checkLikeToRegex(true, "abc", "abc"); @@ -1103,8 +1102,8 @@ private String mm(int majorVersion, int minorVersion) { } private void checkLikeToRegex(boolean b, String pattern, String abc) { - final Pattern regex = CalciteMetaImpl.likeToRegex(Meta.Pat.of(pattern)); - assertThat(b, is(regex.matcher(abc).matches())); + final Pattern regex = LikePattern.likeToRegex(pattern); + assertTrue(b == regex.matcher(abc).matches()); } /** Tests driver's implementation of {@link DatabaseMetaData#getColumns}, @@ -1218,7 +1217,7 @@ private void checkResultSetMetaData(Connection connection, String sql) connection.unwrap(CalciteConnection.class); final SchemaPlus rootSchema = calciteConnection.getRootSchema(); final SchemaPlus foodmart = - requireNonNull(rootSchema.getSubSchema("foodmart")); + requireNonNull(rootSchema.subSchemas().get("foodmart")); rootSchema.add("foodmart2", new CloneSchema(foodmart)); Statement statement = connection.createStatement(); ResultSet resultSet = @@ -1236,10 +1235,10 @@ private void checkResultSetMetaData(Connection connection, String sql) final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class); final SchemaPlus rootSchema = calciteConnection.getRootSchema(); - final SchemaPlus foodmart = rootSchema.getSubSchema("foodmart"); + final SchemaPlus foodmart = rootSchema.subSchemas().get("foodmart"); assertThat(foodmart, notNullValue()); final JdbcTable timeByDay = - requireNonNull((JdbcTable) foodmart.getTable("time_by_day")); + requireNonNull((JdbcTable) foodmart.tables().get("time_by_day")); final int rows = timeByDay.scan(DataContexts.of(calciteConnection, rootSchema)).count(); assertThat(rows, OrderingComparison.greaterThan(0)); } @@ -7489,7 +7488,7 @@ private void checkGetTimestamp(Connection con) throws SQLException { aSchema.setCacheEnabled(true); // explicit should win implicit. - assertThat(aSchema.getSubSchemaNames(), hasSize(1)); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(1)); } @Test void testSimpleCalciteSchema() { @@ -7510,13 +7509,13 @@ private void checkGetTimestamp(Connection con) throws SQLException { // add implicit schema "/a/c" aSubSchemaMap.put("c", new AbstractSchema()); - assertThat(aSchema.getSubSchema("c"), notNullValue()); - assertThat(aSchema.getSubSchema("b"), notNullValue()); + assertThat(aSchema.subSchemas().get("c"), notNullValue()); + assertThat(aSchema.subSchemas().get("b"), notNullValue()); // add implicit schema "/a/b" aSubSchemaMap.put("b", new AbstractSchema()); // explicit should win implicit. - assertThat(aSchema.getSubSchemaNames(), hasSize(2)); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(2)); } @Test void testCaseSensitiveConfigurableSimpleCalciteSchema() { @@ -7577,7 +7576,7 @@ private void checkGetTimestamp(Connection con) throws SQLException { }); // add view definition final String viewName = "V"; - final SchemaPlus a = rootSchema.getSubSchema("a"); + final SchemaPlus a = rootSchema.subSchemas().get("a"); assertThat(a, notNullValue()); final org.apache.calcite.schema.Function view = ViewTable.viewMacro(a, @@ -7615,33 +7614,33 @@ private void checkGetTimestamp(Connection con) throws SQLException { } }); aSchema.setCacheEnabled(true); - assertThat(aSchema.getSubSchemaNames(), hasSize(0)); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(0)); // first call, to populate the cache - assertThat(aSchema.getSubSchemaNames(), hasSize(0)); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(0)); // create schema "/a/b1". Appears only when we disable caching. aSubSchemaMap.put("b1", new AbstractSchema()); - assertThat(aSchema.getSubSchemaNames(), hasSize(0)); - assertThat(aSchema.getSubSchema("b1"), nullValue()); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(0)); + assertThat(aSchema.subSchemas().get("b1"), nullValue()); aSchema.setCacheEnabled(false); - assertThat(aSchema.getSubSchemaNames(), hasSize(1)); - assertThat(aSchema.getSubSchema("b1"), notNullValue()); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(1)); + assertThat(aSchema.subSchemas().get("b1"), notNullValue()); // create schema "/a/b2". Appears immediately, because caching is disabled. aSubSchemaMap.put("b2", new AbstractSchema()); - assertThat(aSchema.getSubSchemaNames(), hasSize(2)); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(2)); // an explicit sub-schema appears immediately, even if caching is enabled aSchema.setCacheEnabled(true); - assertThat(aSchema.getSubSchemaNames(), hasSize(2)); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(2)); aSchema.add("b3", new AbstractSchema()); // explicit aSubSchemaMap.put("b4", new AbstractSchema()); // implicit - assertThat(aSchema.getSubSchemaNames(), hasSize(3)); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(3)); aSchema.setCacheEnabled(false); - assertThat(aSchema.getSubSchemaNames(), hasSize(4)); - for (String name : aSchema.getSubSchemaNames()) { - assertThat(aSchema.getSubSchema(name), notNullValue()); + assertThat(aSchema.subSchemas().getNames(LikePattern.any()), hasSize(4)); + for (String name : aSchema.subSchemas().getNames(LikePattern.any())) { + assertThat(aSchema.subSchemas().get(name), notNullValue()); } // create schema "/a2" @@ -7652,21 +7651,21 @@ private void checkGetTimestamp(Connection con) throws SQLException { } }); a2Schema.setCacheEnabled(true); - assertThat(a2Schema.getSubSchemaNames(), hasSize(0)); + assertThat(a2Schema.subSchemas().getNames(LikePattern.any()), hasSize(0)); // create schema "/a2/b3". Change not visible since caching is enabled. a2SubSchemaMap.put("b3", new AbstractSchema()); - assertThat(a2Schema.getSubSchemaNames(), hasSize(0)); + assertThat(a2Schema.subSchemas().getNames(LikePattern.any()), hasSize(0)); Thread.sleep(1); - assertThat(a2Schema.getSubSchemaNames(), hasSize(0)); + assertThat(a2Schema.subSchemas().getNames(LikePattern.any()), hasSize(0)); // Change visible after we turn off caching. a2Schema.setCacheEnabled(false); - assertThat(a2Schema.getSubSchemaNames(), hasSize(1)); + assertThat(a2Schema.subSchemas().getNames(LikePattern.any()), hasSize(1)); a2SubSchemaMap.put("b4", new AbstractSchema()); - assertThat(a2Schema.getSubSchemaNames(), hasSize(2)); - for (String name : aSchema.getSubSchemaNames()) { - assertThat(aSchema.getSubSchema(name), notNullValue()); + assertThat(a2Schema.subSchemas().getNames(LikePattern.any()), hasSize(2)); + for (String name : aSchema.subSchemas().getNames(LikePattern.any())) { + assertThat(aSchema.subSchemas().get(name), notNullValue()); } // add tables and retrieve with various case sensitivities @@ -7676,7 +7675,7 @@ private void checkGetTimestamp(Connection con) throws SQLException { a2Schema.add("TABLE1", table); a2Schema.add("tabLe1", table); a2Schema.add("tabLe2", table); - assertThat(a2Schema.getTableNames(), hasSize(4)); + assertThat(a2Schema.tables().getNames(LikePattern.any()), hasSize(4)); final CalciteSchema a2CalciteSchema = CalciteSchema.from(a2Schema); assertThat(a2CalciteSchema.getTable("table1", true), notNullValue()); assertThat(a2CalciteSchema.getTable("table1", false), notNullValue()); @@ -8809,7 +8808,7 @@ public FoodmartJdbcSchema(DataSource dataSource, SqlDialect dialect, } public final Table customer = - requireNonNull(getTable("customer")); + requireNonNull(tables().get("customer")); } public static class Customer { diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java index 1a16626150c..7c3aa99d934 100644 --- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java +++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java @@ -206,8 +206,8 @@ private static CalciteAssert.AssertThat modelWithLattices( .doWithConnection(c -> { final SchemaPlus schema = c.getRootSchema(); final SchemaPlus adhoc = - requireNonNull(schema.getSubSchema("adhoc")); - assertThat(adhoc.getTableNames().contains("EMPLOYEES"), is(true)); + requireNonNull(schema.subSchemas().get("adhoc")); + assertThat(adhoc.tables().get("EMPLOYEES") != null, is(true)); final CalciteSchema adhocSchema = requireNonNull(adhoc.unwrap(CalciteSchema.class)); final Map.Entry entry = @@ -238,8 +238,8 @@ private static CalciteAssert.AssertThat modelWithLattices( .doWithConnection(c -> { final SchemaPlus schema = c.getRootSchema(); final SchemaPlus adhoc = - requireNonNull(schema.getSubSchema("adhoc")); - assertThat(adhoc.getTableNames().contains("EMPLOYEES"), is(true)); + requireNonNull(schema.subSchemas().get("adhoc")); + assertThat(adhoc.tables().get("EMPLOYEES") != null, is(true)); final CalciteSchema adhocSchema = requireNonNull(adhoc.unwrap(CalciteSchema.class)); final Map.Entry entry = @@ -269,8 +269,8 @@ private static CalciteAssert.AssertThat modelWithLattices( .doWithConnection(c -> { final SchemaPlus schema = c.getRootSchema(); final SchemaPlus adhoc = - requireNonNull(schema.getSubSchema("adhoc")); - assertThat(adhoc.getTableNames().contains("EMPLOYEES"), is(true)); + requireNonNull(schema.subSchemas().get("adhoc")); + assertThat(adhoc.tables().get("EMPLOYEES") != null, is(true)); final CalciteSchema adhocSchema = requireNonNull(adhoc.unwrap(CalciteSchema.class)); final Map.Entry entry = diff --git a/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java b/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java index 861692d9b27..1543da0baa3 100644 --- a/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java +++ b/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java @@ -25,6 +25,7 @@ import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.jdbc.Driver; import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.lookup.LikePattern; import org.apache.calcite.test.schemata.hr.HrSchema; import org.apache.commons.dbcp2.BasicDataSource; @@ -105,7 +106,7 @@ class MultiJdbcSchemaJoinTest { dataSource.setPassword(""); final JdbcCatalogSchema schema = JdbcCatalogSchema.create(null, "", dataSource, "PUBLIC"); - assertThat(schema.getSubSchemaNames(), + assertThat(schema.subSchemas().getNames(LikePattern.any()), is(Sets.newHashSet("INFORMATION_SCHEMA", "PUBLIC", "SYSTEM_LOBS"))); final CalciteSchema rootSchema0 = CalciteSchema.createRootSchema(false, false, "", schema);