Return-Path: X-Original-To: apmail-calcite-commits-archive@www.apache.org Delivered-To: apmail-calcite-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 944D81748F for ; Wed, 13 May 2015 19:06:51 +0000 (UTC) Received: (qmail 69179 invoked by uid 500); 13 May 2015 19:06:51 -0000 Delivered-To: apmail-calcite-commits-archive@calcite.apache.org Received: (qmail 69150 invoked by uid 500); 13 May 2015 19:06:51 -0000 Mailing-List: contact commits-help@calcite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@calcite.incubator.apache.org Delivered-To: mailing list commits@calcite.incubator.apache.org Received: (qmail 69141 invoked by uid 99); 13 May 2015 19:06:51 -0000 Received: from Unknown (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 13 May 2015 19:06:51 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id DD8C31A2B44 for ; Wed, 13 May 2015 19:06:50 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.791 X-Spam-Level: * X-Spam-Status: No, score=1.791 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, T_RP_MATCHES_RCVD=-0.01, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-eu-west.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id 1HS9TwVaLQpo for ; Wed, 13 May 2015 19:06:36 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with SMTP id A9D2524F8D for ; Wed, 13 May 2015 19:06:34 +0000 (UTC) Received: (qmail 68699 invoked by uid 99); 13 May 2015 19:06:33 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 13 May 2015 19:06:33 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7B635E35A2; Wed, 13 May 2015 19:06:33 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jhyde@apache.org To: commits@calcite.incubator.apache.org Date: Wed, 13 May 2015 19:06:44 -0000 Message-Id: <9ebc4d0d419d4ecf8e796c46b4088a76@git.apache.org> In-Reply-To: <9f476fabe83041bd9a1234a822c0f46a@git.apache.org> References: <9f476fabe83041bd9a1234a822c0f46a@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [13/13] incubator-calcite git commit: [CALCITE-505] Support modifiable view [CALCITE-505] Support modifiable view Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/bc45a2c0 Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/bc45a2c0 Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/bc45a2c0 Branch: refs/heads/master Commit: bc45a2c047349826e22641e82468a2d4dad948e3 Parents: aee32bc Author: Julian Hyde Authored: Thu Mar 19 09:56:09 2015 -0700 Committer: Julian Hyde Committed: Tue May 12 14:18:10 2015 -0700 ---------------------------------------------------------------------- .../adapter/enumerable/RexToLixTranslator.java | 3 + .../org/apache/calcite/jdbc/CalcitePrepare.java | 61 +++-- .../java/org/apache/calcite/model/JsonView.java | 14 ++ .../org/apache/calcite/model/ModelHandler.java | 3 +- .../org/apache/calcite/plan/RelOptUtil.java | 48 +++- .../calcite/prepare/CalcitePrepareImpl.java | 149 ++++++++++++- .../apache/calcite/prepare/RelOptTableImpl.java | 146 +++++++++++- .../java/org/apache/calcite/rex/RexCopier.java | 18 +- .../apache/calcite/runtime/CalciteResource.java | 9 + .../apache/calcite/schema/ModifiableTable.java | 2 + .../apache/calcite/schema/ModifiableView.java | 69 ++++++ .../java/org/apache/calcite/schema/Path.java | 38 ++++ .../java/org/apache/calcite/schema/Schemas.java | 94 +++++++- .../schema/impl/MaterializedViewTable.java | 2 +- .../apache/calcite/schema/impl/ViewTable.java | 100 +++++++-- .../calcite/sql2rel/SqlToRelConverter.java | 98 ++++++-- .../calcite/runtime/CalciteResource.properties | 3 + .../apache/calcite/sql/test/SqlAdvisorTest.java | 1 + .../org/apache/calcite/test/CalciteAssert.java | 6 +- .../calcite/test/JdbcFrontLinqBackTest.java | 93 ++++---- .../java/org/apache/calcite/test/JdbcTest.java | 223 ++++++++++++++++++- .../apache/calcite/test/MockCatalogReader.java | 223 +++++++++++++++---- .../calcite/test/ReflectiveSchemaTest.java | 10 +- .../calcite/test/SqlToRelConverterTest.java | 16 ++ .../apache/calcite/test/SqlValidatorTest.java | 6 + .../calcite/test/SqlToRelConverterTest.xml | 40 ++++ doc/model.md | 26 ++- .../calcite/linq4j/tree/ConstantExpression.java | 27 ++- .../apache/calcite/linq4j/tree/Expressions.java | 6 +- .../apache/calcite/linq4j/test/Linq4jTest.java | 33 +++ 30 files changed, 1371 insertions(+), 196 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java index 9e7502d..eed3d9d 100644 --- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java +++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java @@ -586,6 +586,9 @@ public class RexToLixTranslator { final Object value2; switch (literal.getType().getSqlTypeName()) { case DECIMAL: + if (javaClass == float.class) { + return Expressions.constant(value, javaClass); + } assert javaClass == BigDecimal.class; return Expressions.new_(BigDecimal.class, Expressions.constant(value.toString())); http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java index acbdfe9..c792ed2 100644 --- a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java +++ b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java @@ -33,14 +33,19 @@ import org.apache.calcite.prepare.CalcitePrepareImpl; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rex.RexNode; import org.apache.calcite.runtime.ArrayBindable; import org.apache.calcite.runtime.Bindable; +import org.apache.calcite.schema.Table; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.util.ImmutableIntList; import org.apache.calcite.util.Stacks; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.collect.ImmutableList; + import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Type; @@ -61,7 +66,7 @@ public interface CalcitePrepare { ThreadLocal> THREAD_CONTEXT_STACK = new ThreadLocal>() { @Override protected ArrayList initialValue() { - return new ArrayList(); + return new ArrayList<>(); } }; @@ -69,6 +74,16 @@ public interface CalcitePrepare { ConvertResult convert(Context context, String sql); + /** Analyzes a view. + * + * @param context Context + * @param sql View SQL + * @param fail Whether to fail (and throw a descriptive error message) if the + * view is not modifiable + * @return Result of analyzing the view + */ + AnalyzeViewResult analyzeView(Context context, String sql, boolean fail); + CalciteSignature prepareSql( Context context, String sql, @@ -97,7 +112,7 @@ public interface CalcitePrepare { } /** Callback to register Spark as the main engine. */ - public interface SparkHandler { + interface SparkHandler { RelNode flattenTypes(RelOptPlanner planner, RelNode rootRel, boolean restructure); @@ -118,9 +133,11 @@ public interface CalcitePrepare { /** Namespace that allows us to define non-abstract methods inside an * interface. */ - public static class Dummy { + class Dummy { private static SparkHandler sparkHandler; + private Dummy() {} + /** Returns a spark handler. Returns a trivial handler, for which * {@link SparkHandler#enabled()} returns {@code false}, if {@code enable} * is {@code false} or if Spark is not on the class path. Never returns @@ -140,13 +157,10 @@ public interface CalcitePrepare { return (CalcitePrepare.SparkHandler) method.invoke(null); } catch (ClassNotFoundException e) { return new TrivialSparkHandler(); - } catch (IllegalAccessException e) { - throw new RuntimeException(e); - } catch (ClassCastException e) { - throw new RuntimeException(e); - } catch (NoSuchMethodException e) { - throw new RuntimeException(e); - } catch (InvocationTargetException e) { + } catch (IllegalAccessException + | ClassCastException + | InvocationTargetException + | NoSuchMethodException e) { throw new RuntimeException(e); } } @@ -189,7 +203,7 @@ public interface CalcitePrepare { } /** The result of parsing and validating a SQL query. */ - public static class ParseResult { + class ParseResult { public final CalcitePrepareImpl prepare; public final String sql; // for debug public final SqlNode sqlNode; @@ -210,7 +224,7 @@ public interface CalcitePrepare { /** The result of parsing and validating a SQL query and converting it to * relational algebra. */ - public static class ConvertResult extends ParseResult { + class ConvertResult extends ParseResult { public final RelNode relNode; public ConvertResult(CalcitePrepareImpl prepare, SqlValidator validator, @@ -220,10 +234,31 @@ public interface CalcitePrepare { } } + /** The result of analyzing a view. */ + class AnalyzeViewResult extends ConvertResult { + /** Not null if and only if the view is modifiable. */ + public final Table table; + public final ImmutableList tablePath; + public final RexNode constraint; + public final ImmutableIntList columnMapping; + + public AnalyzeViewResult(CalcitePrepareImpl prepare, + SqlValidator validator, String sql, SqlNode sqlNode, + RelDataType rowType, RelNode relNode, Table table, + ImmutableList tablePath, RexNode constraint, + ImmutableIntList columnMapping) { + super(prepare, validator, sql, sqlNode, rowType, relNode); + this.table = table; + this.tablePath = tablePath; + this.constraint = constraint; + this.columnMapping = columnMapping; + } + } + /** The result of preparing a query. It gives the Avatica driver framework * the information it needs to create a prepared statement, or to execute a * statement directly, without an explicit prepare step. */ - public static class CalciteSignature extends Meta.Signature { + class CalciteSignature extends Meta.Signature { @JsonIgnore public final RelDataType rowType; private final int maxRowCount; private final Bindable bindable; http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/model/JsonView.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/model/JsonView.java b/core/src/main/java/org/apache/calcite/model/JsonView.java index f482765..48e1cf3 100644 --- a/core/src/main/java/org/apache/calcite/model/JsonView.java +++ b/core/src/main/java/org/apache/calcite/model/JsonView.java @@ -31,6 +31,20 @@ public class JsonView extends JsonTable { * to current schema. */ public List path; + /** Whether this view should allow INSERT requests. + * + *

The values have the following meanings: + *

    + *
  • If true, Calcite throws an error when validating the schema if the + * view is not modifiable. + *
  • If null, Calcite deduces whether the view is modifiable. + *
  • If false, Calcite will not allow inserts. + *
+ * + *

The default value is {@code null}. + */ + public Boolean modifiable; + public void accept(ModelHandler handler) { handler.visit(this); } http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/model/ModelHandler.java ---------------------------------------------------------------------- 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 01ed89e..dfdb901 100644 --- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java +++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java @@ -331,7 +331,8 @@ public class ModelHandler { final SchemaPlus schema = currentMutableSchema("view"); final List path = Util.first(jsonView.path, currentSchemaPath()); schema.add(jsonView.name, - ViewTable.viewMacro(schema, jsonView.getSql(), path)); + ViewTable.viewMacro(schema, jsonView.getSql(), path, + jsonView.modifiable)); } catch (Exception e) { throw new RuntimeException("Error instantiating " + jsonView, e); } http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java index 3deb4e4..6090dd2 100644 --- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java +++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java @@ -92,6 +92,7 @@ import java.util.BitSet; import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; @@ -636,8 +637,8 @@ public abstract class RelOptUtil { final RelNode rel, RelDataType castRowType, boolean rename) { - return createCastRel(rel, castRowType, rename, - RelFactories.DEFAULT_PROJECT_FACTORY); + return createCastRel( + rel, castRowType, rename, RelFactories.DEFAULT_PROJECT_FACTORY); } /** @@ -2033,6 +2034,45 @@ public abstract class RelOptUtil { return left; } + /** Decomposes the WHERE clause of a view into predicates that constraint + * a column to a particular value. + * + *

This method is key to the validation of a modifiable view. Columns that + * are constrained to a single value can be omitted from the + * SELECT clause of a modifiable view. + * + * @param projectMap Mapping from column ordinal to the expression that + * populate that column, to be populated by this method + * @param filters List of remaining filters, to be populated by this method + * @param constraint Constraint to be analyzed + */ + public static void inferViewPredicates(Map projectMap, + List filters, RexNode constraint) { + for (RexNode node : conjunctions(constraint)) { + switch (node.getKind()) { + case EQUALS: + final List operands = ((RexCall) node).getOperands(); + RexNode o0 = operands.get(0); + RexNode o1 = operands.get(1); + if (o0 instanceof RexLiteral) { + o0 = operands.get(1); + o1 = operands.get(0); + } + if (o0.getKind() == SqlKind.CAST) { + o0 = ((RexCall) o0).getOperands().get(0); + } + if (o0 instanceof RexInputRef && o1 instanceof RexLiteral) { + final int index = ((RexInputRef) o0).getIndex(); + if (projectMap.get(index) == null) { + projectMap.put(index, o1); + continue; + } + } + } + filters.add(node); + } + } + /** * Adjusts key values in a list by some fixed amount. * @@ -2623,8 +2663,8 @@ public abstract class RelOptUtil { */ public static RelNode createProject(final RelNode child, final List posList) { - return createProject(RelFactories.DEFAULT_PROJECT_FACTORY, - child, posList); + return createProject( + RelFactories.DEFAULT_PROJECT_FACTORY, child, posList); } /** http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java index 9d00a87..9ebfe8a 100644 --- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java +++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java @@ -64,6 +64,9 @@ import org.apache.calcite.plan.hep.HepProgramBuilder; import org.apache.calcite.plan.volcano.VolcanoPlanner; import org.apache.calcite.rel.RelCollationTraitDef; import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule; import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule; import org.apache.calcite.rel.rules.AggregateStarTableRule; @@ -87,11 +90,13 @@ import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeFactoryImpl; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.apache.calcite.runtime.Bindable; import org.apache.calcite.runtime.Hook; import org.apache.calcite.runtime.Typed; import org.apache.calcite.schema.Schemas; +import org.apache.calcite.schema.Table; import org.apache.calcite.server.CalciteServerStatement; import org.apache.calcite.sql.SqlBinaryOperator; import org.apache.calcite.sql.SqlExplainLevel; @@ -109,6 +114,7 @@ import org.apache.calcite.sql.validate.SqlValidatorImpl; import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.sql2rel.StandardConvertletTable; import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.util.ImmutableIntList; import org.apache.calcite.util.Util; import com.google.common.collect.ImmutableList; @@ -121,10 +127,13 @@ import java.math.BigDecimal; import java.sql.DatabaseMetaData; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; +import static org.apache.calcite.util.Static.RESOURCE; + /** * Shit just got real. * @@ -223,15 +232,21 @@ public class CalcitePrepareImpl implements CalcitePrepare { public ParseResult parse( Context context, String sql) { - return parse_(context, sql, false); + return parse_(context, sql, false, false, false); } public ConvertResult convert(Context context, String sql) { - return (ConvertResult) parse_(context, sql, true); + return (ConvertResult) parse_(context, sql, true, false, false); + } + + public AnalyzeViewResult analyzeView(Context context, String sql, boolean fail) { + return (AnalyzeViewResult) parse_(context, sql, true, true, fail); } - /** Shared implementation for {@link #parse} and {@link #convert}. */ - private ParseResult parse_(Context context, String sql, boolean convert) { + /** Shared implementation for {@link #parse}, {@link #convert} and + * {@link #analyzeView}. */ + private ParseResult parse_(Context context, String sql, boolean convert, + boolean analyze, boolean fail) { final JavaTypeFactory typeFactory = context.getTypeFactory(); CalciteCatalogReader catalogReader = new CalciteCatalogReader( @@ -250,24 +265,144 @@ public class CalcitePrepareImpl implements CalcitePrepare { new CalciteSqlValidator( SqlStdOperatorTable.instance(), catalogReader, typeFactory); SqlNode sqlNode1 = validator.validate(sqlNode); - if (!convert) { - return new ParseResult(this, validator, sql, sqlNode1, - validator.getValidatedNodeType(sqlNode1)); + if (convert) { + return convert_( + context, sql, analyze, fail, catalogReader, validator, sqlNode1); } + return new ParseResult(this, validator, sql, sqlNode1, + validator.getValidatedNodeType(sqlNode1)); + } + + private ParseResult convert_(Context context, String sql, boolean analyze, + boolean fail, CalciteCatalogReader catalogReader, SqlValidator validator, + SqlNode sqlNode1) { + final JavaTypeFactory typeFactory = context.getTypeFactory(); final Convention resultConvention = ENABLE_BINDABLE ? BindableConvention.INSTANCE : EnumerableConvention.INSTANCE; final HepPlanner planner = new HepPlanner(new HepProgramBuilder().build()); + planner.addRelTraitDef(ConventionTraitDef.INSTANCE); final CalcitePreparingStmt preparingStmt = new CalcitePreparingStmt(this, context, catalogReader, typeFactory, context.getRootSchema(), null, planner, resultConvention); final SqlToRelConverter converter = preparingStmt.getSqlToRelConverter(validator, catalogReader); + if (analyze) { + converter.enableTableAccessConversion(false); + } final RelNode relNode = converter.convertQuery(sqlNode1, false, true); + if (analyze) { + return analyze_(validator, sql, sqlNode1, relNode, fail); + } return new ConvertResult(this, validator, sql, sqlNode1, validator.getValidatedNodeType(sqlNode1), relNode); } + private AnalyzeViewResult analyze_(SqlValidator validator, String sql, + SqlNode sqlNode, RelNode rel, boolean fail) { + final RexBuilder rexBuilder = rel.getCluster().getRexBuilder(); + final RelNode viewRel = rel; + Project project; + if (rel instanceof Project) { + project = (Project) rel; + rel = project.getInput(); + } else { + project = null; + } + Filter filter; + if (rel instanceof Filter) { + filter = (Filter) rel; + rel = filter.getInput(); + } else { + filter = null; + } + TableScan scan; + if (rel instanceof TableScan) { + scan = (TableScan) rel; + } else { + scan = null; + } + if (scan == null) { + if (fail) { + throw validator.newValidationError(sqlNode, + RESOURCE.modifiableViewMustBeBasedOnSingleTable()); + } + return new AnalyzeViewResult(this, validator, sql, sqlNode, + validator.getValidatedNodeType(sqlNode), rel, null, null, null, + null); + } + final RelOptTable targetRelTable = scan.getTable(); + final RelDataType targetRowType = targetRelTable.getRowType(); + final Table table = targetRelTable.unwrap(Table.class); + final List tablePath = targetRelTable.getQualifiedName(); + assert table != null; + List columnMapping; + final Map projectMap = new HashMap<>(); + if (project == null) { + columnMapping = ImmutableIntList.range(0, targetRowType.getFieldCount()); + } else { + columnMapping = new ArrayList<>(); + for (Ord node : Ord.zip(project.getProjects())) { + if (node.e instanceof RexInputRef) { + RexInputRef rexInputRef = (RexInputRef) node.e; + int index = rexInputRef.getIndex(); + if (projectMap.get(index) != null) { + if (fail) { + throw validator.newValidationError(sqlNode, + RESOURCE.moreThanOneMappedColumn( + targetRowType.getFieldList().get(index).getName(), + Util.last(tablePath))); + } + return new AnalyzeViewResult(this, validator, sql, sqlNode, + validator.getValidatedNodeType(sqlNode), rel, null, null, null, + null); + } + projectMap.put(index, rexBuilder.makeInputRef(viewRel, node.i)); + columnMapping.add(index); + } else { + columnMapping.add(-1); + } + } + } + final RexNode constraint; + if (filter != null) { + constraint = filter.getCondition(); + } else { + constraint = rexBuilder.makeLiteral(true); + } + final List filters = new ArrayList<>(); + RelOptUtil.inferViewPredicates(projectMap, filters, constraint); + + // Check that all columns that are not projected have a constant value + for (RelDataTypeField field : targetRowType.getFieldList()) { + final int x = columnMapping.indexOf(field.getIndex()); + if (x >= 0) { + assert Util.skip(columnMapping, x + 1).indexOf(field.getIndex()) < 0 + : "column projected more than once; should have checked above"; + continue; // target column is projected + } + if (projectMap.get(field.getIndex()) != null) { + continue; // constant expression + } + if (field.getType().isNullable()) { + continue; // don't need expression for nullable columns; NULL suffices + } + if (fail) { + throw validator.newValidationError(sqlNode, + RESOURCE.noValueSuppliedForViewColumn(field.getName(), + Util.last(tablePath))); + } + return new AnalyzeViewResult(this, validator, sql, sqlNode, + validator.getValidatedNodeType(sqlNode), rel, null, null, null, + null); + } + + return new AnalyzeViewResult(this, validator, sql, sqlNode, + validator.getValidatedNodeType(sqlNode), rel, table, + ImmutableList.copyOf(tablePath), + constraint, ImmutableIntList.copyOf(columnMapping)); + } + /** Factory method for default SQL parser. */ protected SqlParser createParser(String sql) { return createParser(sql, createParserConfig()); http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java ---------------------------------------------------------------------- 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 8925ede..9919cec 100644 --- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java +++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java @@ -19,6 +19,7 @@ package org.apache.calcite.prepare; import org.apache.calcite.adapter.enumerable.EnumerableTableScan; import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.materialize.Lattice; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptSchema; import org.apache.calcite.plan.RelOptTable; @@ -32,9 +33,13 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.schema.ExtensibleTable; import org.apache.calcite.schema.FilterableTable; +import org.apache.calcite.schema.ModifiableTable; +import org.apache.calcite.schema.Path; import org.apache.calcite.schema.ProjectableFilterableTable; import org.apache.calcite.schema.QueryableTable; import org.apache.calcite.schema.ScannableTable; +import org.apache.calcite.schema.Schema; +import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.StreamableTable; import org.apache.calcite.schema.Table; @@ -43,6 +48,7 @@ import org.apache.calcite.sql.SqlAccessType; import org.apache.calcite.sql.validate.SqlModality; import org.apache.calcite.sql.validate.SqlMonotonicity; import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.Pair; import org.apache.calcite.util.Util; import com.google.common.base.Function; @@ -50,7 +56,9 @@ import com.google.common.base.Functions; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import java.util.Collection; import java.util.List; +import java.util.Set; /** * Implementation of {@link org.apache.calcite.plan.RelOptTable}. @@ -99,6 +107,15 @@ public class RelOptTableImpl implements Prepare.PreparingTable { } public static RelOptTableImpl create(RelOptSchema schema, RelDataType rowType, + Table table, Path path) { + final SchemaPlus schemaPlus = MySchemaPlus.create(path); + Function expressionFunction = + getClassExpressionFunction(schemaPlus, Util.last(path).left, table); + return new RelOptTableImpl(schema, rowType, Pair.left(path), table, + expressionFunction, table.getStatistic().getRowCount()); + } + + public static RelOptTableImpl create(RelOptSchema schema, RelDataType rowType, final CalciteSchema.TableEntry tableEntry, Double rowCount) { final Table table = tableEntry.getTable(); Function expressionFunction = @@ -108,13 +125,18 @@ public class RelOptTableImpl implements Prepare.PreparingTable { } private static Function getClassExpressionFunction( - final CalciteSchema.TableEntry tableEntry, final Table table) { + CalciteSchema.TableEntry tableEntry, Table table) { + return getClassExpressionFunction(tableEntry.schema.plus(), tableEntry.name, + table); + } + + private static Function getClassExpressionFunction( + final SchemaPlus schema, final String tableName, final Table table) { if (table instanceof QueryableTable) { final QueryableTable queryableTable = (QueryableTable) table; return new Function() { public Expression apply(Class clazz) { - return queryableTable.getExpression(tableEntry.schema.plus(), - tableEntry.name, clazz); + return queryableTable.getExpression(schema, tableName, clazz); } }; } else if (table instanceof ScannableTable @@ -122,14 +144,12 @@ public class RelOptTableImpl implements Prepare.PreparingTable { || table instanceof ProjectableFilterableTable) { return new Function() { public Expression apply(Class clazz) { - return Schemas.tableExpression(tableEntry.schema.plus(), - Object[].class, - tableEntry.name, + return Schemas.tableExpression(schema, Object[].class, tableName, table.getClass()); } }; } else if (table instanceof StreamableTable) { - return getClassExpressionFunction(tableEntry, + return getClassExpressionFunction(schema, tableName, ((StreamableTable) table).stream()); } else { return new Function() { @@ -145,7 +165,8 @@ public class RelOptTableImpl implements Prepare.PreparingTable { RelDataType rowType, Table table) { assert table instanceof TranslatableTable - || table instanceof ScannableTable; + || table instanceof ScannableTable + || table instanceof ModifiableTable; return new RelOptTableImpl(schema, rowType, ImmutableList.of(), table, null, null); } @@ -299,6 +320,115 @@ public class RelOptTableImpl implements Prepare.PreparingTable { public SqlAccessType getAllowedAccess() { return SqlAccessType.ALL; } + + /** Im0plementation of {@link SchemaPlus} that wraps a regular schema and knows + * its name and parent. + * + *

It is read-only, and functionality is limited in other ways, it but + * allows table expressions to be genenerated. */ + private static class MySchemaPlus implements SchemaPlus { + private final SchemaPlus parent; + private final String name; + private final Schema schema; + + public MySchemaPlus(SchemaPlus parent, String name, Schema schema) { + this.parent = parent; + this.name = name; + this.schema = schema; + } + + public static MySchemaPlus create(Path path) { + final Pair pair = Util.last(path); + final SchemaPlus parent; + if (path.size() == 1) { + parent = null; + } else { + parent = create(path.parent()); + } + return new MySchemaPlus(parent, pair.left, pair.right); + } + + @Override public SchemaPlus getParentSchema() { + return parent; + } + + @Override public String getName() { + return name; + } + + @Override public SchemaPlus getSubSchema(String name) { + final Schema subSchema = schema.getSubSchema(name); + return subSchema == null ? null : new MySchemaPlus(this, name, subSchema); + } + + @Override public SchemaPlus add(String name, Schema schema) { + throw new UnsupportedOperationException(); + } + + @Override public void add(String name, Table table) { + throw new UnsupportedOperationException(); + } + + @Override public void add(String name, + org.apache.calcite.schema.Function function) { + throw new UnsupportedOperationException(); + } + + @Override public void add(String name, Lattice lattice) { + throw new UnsupportedOperationException(); + } + + @Override public boolean isMutable() { + return schema.isMutable(); + } + + @Override public T unwrap(Class clazz) { + return null; + } + + @Override public void setPath(ImmutableList> path) { + throw new UnsupportedOperationException(); + } + + @Override public void setCacheEnabled(boolean cache) { + throw new UnsupportedOperationException(); + } + + @Override public boolean isCacheEnabled() { + return false; + } + + @Override public Table getTable(String name) { + return schema.getTable(name); + } + + @Override public Set getTableNames() { + return schema.getTableNames(); + } + + @Override public Collection + getFunctions(String name) { + return schema.getFunctions(name); + } + + @Override public Set getFunctionNames() { + return schema.getFunctionNames(); + } + + @Override public Set getSubSchemaNames() { + return schema.getSubSchemaNames(); + } + + @Override public Expression getExpression(SchemaPlus parentSchema, + String name) { + return schema.getExpression(parentSchema, name); + } + + @Override public boolean contentsHaveChangedSince(long lastCheck, + long now) { + return schema.contentsHaveChangedSince(lastCheck, now); + } + } } // End RelOptTableImpl.java http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/rex/RexCopier.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/rex/RexCopier.java b/core/src/main/java/org/apache/calcite/rex/RexCopier.java index 708a59a..3563cdd 100644 --- a/core/src/main/java/org/apache/calcite/rex/RexCopier.java +++ b/core/src/main/java/org/apache/calcite/rex/RexCopier.java @@ -16,6 +16,8 @@ */ package org.apache.calcite.rex; +import org.apache.calcite.rel.type.RelDataType; + /** * Shuttle which creates a deep copy of a Rex expression. * @@ -45,6 +47,10 @@ class RexCopier extends RexShuttle { //~ Methods ---------------------------------------------------------------- + private RelDataType copy(RelDataType type) { + return builder.getTypeFactory().copyType(type); + } + public RexNode visitOver(RexOver over) { throw new UnsupportedOperationException(); } @@ -55,8 +61,7 @@ class RexCopier extends RexShuttle { public RexNode visitCall(final RexCall call) { final boolean[] update = null; - return builder.makeCall( - builder.getTypeFactory().copyType(call.getType()), + return builder.makeCall(copy(call.getType()), call.getOperator(), visitList(call.getOperands(), update)); } @@ -66,13 +71,12 @@ class RexCopier extends RexShuttle { } public RexNode visitFieldAccess(RexFieldAccess fieldAccess) { - return builder.makeFieldAccess( - fieldAccess.getReferenceExpr().accept(this), + return builder.makeFieldAccess(fieldAccess.getReferenceExpr().accept(this), fieldAccess.getField().getIndex()); } public RexNode visitInputRef(RexInputRef inputRef) { - throw new UnsupportedOperationException(); + return builder.makeInputRef(copy(inputRef.getType()), inputRef.getIndex()); } public RexNode visitLocalRef(RexLocalRef localRef) { @@ -80,9 +84,7 @@ class RexCopier extends RexShuttle { } public RexNode visitLiteral(RexLiteral literal) { - return new RexLiteral( - literal.getValue(), - builder.getTypeFactory().copyType(literal.getType()), + return new RexLiteral(literal.getValue(), copy(literal.getType()), literal.getTypeName()); } http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java index 1e8badf..ee3cb5a 100644 --- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java +++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java @@ -573,6 +573,15 @@ public interface CalciteResource { @BaseMessage("Cannot stream VALUES") ExInst cannotStreamValues(); + + @BaseMessage("Modifiable view must be based on a single table") + ExInst modifiableViewMustBeBasedOnSingleTable(); + + @BaseMessage("View is not modifiable. More than one expression maps to column ''{0}'' of base table ''{1}''") + ExInst moreThanOneMappedColumn(String columnName, String tableName); + + @BaseMessage("View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}''") + ExInst noValueSuppliedForViewColumn(String columnName, String tableName); } // End CalciteResource.java http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java b/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java index 85b2a87..ce0f363 100644 --- a/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java +++ b/core/src/main/java/org/apache/calcite/schema/ModifiableTable.java @@ -30,6 +30,8 @@ import java.util.List; * *

NOTE: The current API is inefficient and experimental. It will change * without notice.

+ * + * @see ModifiableView */ public interface ModifiableTable extends QueryableTable { /** Returns the modifiable collection. http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/ModifiableView.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/schema/ModifiableView.java b/core/src/main/java/org/apache/calcite/schema/ModifiableView.java new file mode 100644 index 0000000..c1a2d81 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/ModifiableView.java @@ -0,0 +1,69 @@ +/* + * 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; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.util.ImmutableIntList; + +/** + * A modifiable view onto {@link ModifiableTable}. + * + *

It describes how its columns map onto the underlying table's columns, + * and any constraints that incoming rows must satisfy. + * + *

For example, given + * + *

+ *   CREATE TABLE emps (empno INTEGER, gender VARCHAR(1), deptno INTEGER);
+ *   CREATE VIEW female_emps AS
+ *     SELECT empno, deptno FROM emps WHERE gender = 'F';
+ * 
+ * + * constraint is {@code $1 = 'F'} + * and column mapping is {@code [0, 2]}. + * + *

NOTE: The current API is inefficient and experimental. It will change + * without notice.

+ */ +public interface ModifiableView extends Table { + /** Returns a constraint that each candidate row must satisfy. + * + *

Never null; if there is no constraint, returns "true". + * + * @param rexBuilder Rex builder + * @param tableRowType Row type of the table that this view maps onto + */ + RexNode getConstraint(RexBuilder rexBuilder, RelDataType tableRowType); + + /** Returns the column mapping onto another table. + * + * {@code mapping[i]} contains the column of the underlying table that the + * {@code i}th column of the view comes from, or -1 if it is based on an + * expression. + */ + ImmutableIntList getColumnMapping(); + + /** Returns the underlying table. */ + Table getTable(); + + /** Returns the full path of the underlying table. */ + Path getTablePath(); +} + +// End ModifiableView.java http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/Path.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/schema/Path.java b/core/src/main/java/org/apache/calcite/schema/Path.java new file mode 100644 index 0000000..7c363a0 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/schema/Path.java @@ -0,0 +1,38 @@ +/* + * 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; + +import org.apache.calcite.util.Pair; + +import java.util.List; +import java.util.RandomAccess; + +/** + * Path from a root schema to a particular object (schema, table, function). + * + *

Examples: + *

    + *
  • The root schema has a single element [(root, "")]. + *
  • A direct child "foo" of the root schema has a two elements + * [(root, ""), (child, "foo")]. + *
+ */ +public interface Path extends List>, RandomAccess { + Path parent(); +} + +// End Path.java http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/Schemas.java ---------------------------------------------------------------------- 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 63d0e4e..b634233 100644 --- a/core/src/main/java/org/apache/calcite/schema/Schemas.java +++ b/core/src/main/java/org/apache/calcite/schema/Schemas.java @@ -39,6 +39,7 @@ import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.type.SqlTypeUtil; import org.apache.calcite.util.BuiltInMethod; +import org.apache.calcite.util.Pair; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -47,6 +48,7 @@ import com.google.common.collect.Lists; import java.lang.reflect.Type; import java.sql.Connection; +import java.util.AbstractList; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -92,8 +94,7 @@ public final class Schemas { String name, Collection functionEntries, List argumentTypes) { - final List matches = - new ArrayList(); + final List matches = new ArrayList<>(); for (CalciteSchema.FunctionEntry entry : functionEntries) { if (matches(typeFactory, entry.getFunction(), argumentTypes)) { matches.add(entry); @@ -285,9 +286,10 @@ public final class Schemas { final CalciteConnection connection, final CalciteSchema schema, final List schemaPath, final String sql) { final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply(); + final ImmutableMap propValues = + ImmutableMap.of(); final CalcitePrepare.Context context = - makeContext(connection, schema, schemaPath, - ImmutableMap.of()); + makeContext(connection, schema, schemaPath, propValues); CalcitePrepare.Dummy.push(context); try { return prepare.parse(context, sql); @@ -302,9 +304,10 @@ public final class Schemas { final CalciteConnection connection, final CalciteSchema schema, final List schemaPath, final String sql) { final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply(); + final ImmutableMap propValues = + ImmutableMap.of(); final CalcitePrepare.Context context = - makeContext(connection, schema, schemaPath, - ImmutableMap.of()); + makeContext(connection, schema, schemaPath, propValues); CalcitePrepare.Dummy.push(context); try { return prepare.convert(context, sql); @@ -313,6 +316,23 @@ public final class Schemas { } } + /** Analyzes a view. For use within Calcite only. */ + public static CalcitePrepare.AnalyzeViewResult analyzeView( + final CalciteConnection connection, final CalciteSchema schema, + final List schemaPath, final String sql, boolean fail) { + final CalcitePrepare prepare = CalcitePrepare.DEFAULT_FACTORY.apply(); + final ImmutableMap propValues = + ImmutableMap.of(); + final CalcitePrepare.Context context = + makeContext(connection, schema, schemaPath, propValues); + CalcitePrepare.Dummy.push(context); + try { + return prepare.analyzeView(context, sql, fail); + } finally { + CalcitePrepare.Dummy.pop(context); + } + } + /** Prepares a SQL query for execution. For use within Calcite only. */ public static CalcitePrepare.CalciteSignature prepare( final CalciteConnection connection, final CalciteSchema schema, @@ -470,6 +490,30 @@ public final class Schemas { return t; } + /** Creates a path with a given list of names starting from a given root + * schema. */ + public static Path path(CalciteSchema rootSchema, Iterable names) { + final ImmutableList.Builder> builder = + ImmutableList.builder(); + Schema schema = rootSchema.schema; + final Iterator iterator = names.iterator(); + if (!iterator.hasNext()) { + return PathImpl.EMPTY; + } + for (;;) { + final String name = iterator.next(); + builder.add(Pair.of(name, schema)); + if (!iterator.hasNext()) { + return path(builder.build()); + } + schema = schema.getSubSchema(name); + } + } + + public static PathImpl path(ImmutableList> build) { + return new PathImpl(build); + } + /** Dummy data context that has no variables. */ private static class DummyDataContext implements DataContext { private final CalciteConnection connection; @@ -497,6 +541,44 @@ public final class Schemas { return map.get(name); } } + + /** Implementation of {@link Path}. */ + private static class PathImpl + extends AbstractList> implements Path { + private final ImmutableList> pairs; + + private static final PathImpl EMPTY = + new PathImpl(ImmutableList.>of()); + + PathImpl(ImmutableList> pairs) { + this.pairs = pairs; + } + + @Override public boolean equals(Object o) { + return this == o + || o instanceof PathImpl + && pairs.equals(((PathImpl) o).pairs); + } + + @Override public int hashCode() { + return pairs.hashCode(); + } + + public Pair get(int index) { + return pairs.get(index); + } + + public int size() { + return pairs.size(); + } + + @Override public Path parent() { + if (pairs.isEmpty()) { + throw new IllegalArgumentException("at root"); + } + return new PathImpl(pairs.subList(0, pairs.size() - 1)); + } + } } // End Schemas.java http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java b/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java index 5810f76..27e873a 100644 --- a/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java +++ b/core/src/main/java/org/apache/calcite/schema/impl/MaterializedViewTable.java @@ -103,7 +103,7 @@ public class MaterializedViewTable extends ViewTable { private MaterializedViewTableMacro(CalciteSchema schema, String viewSql, List viewSchemaPath, String suggestedTableName) { - super(schema, viewSql, viewSchemaPath); + super(schema, viewSql, viewSchemaPath, Boolean.TRUE); this.key = Preconditions.checkNotNull( MaterializationService.instance().defineMaterialization( schema, null, viewSql, schemaPath, suggestedTableName, true)); http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java b/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java index c978a3a..b3a8ff7 100644 --- a/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java +++ b/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java @@ -29,12 +29,18 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeImpl; import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; import org.apache.calcite.schema.FunctionParameter; +import org.apache.calcite.schema.ModifiableView; +import org.apache.calcite.schema.Path; 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.TableMacro; import org.apache.calcite.schema.TranslatableTable; +import org.apache.calcite.util.ImmutableIntList; import org.apache.calcite.util.Util; import com.google.common.collect.ImmutableList; @@ -64,9 +70,33 @@ public class ViewTable } /** Table macro that returns a view. */ + @Deprecated // to be removed before 2.0 public static ViewTableMacro viewMacro(SchemaPlus schema, final String viewSql, final List schemaPath) { - return new ViewTableMacro(CalciteSchema.from(schema), viewSql, schemaPath); + return viewMacro(schema, viewSql, schemaPath, Boolean.TRUE); + } + + /** Table macro that returns a view. + * + * @param schema Schema the view will belong to + * @param viewSql SQL query + * @param schemaPath Path of schema + * @param modifiable Whether view is modifiable, or null to deduce it + */ + public static ViewTableMacro viewMacro(SchemaPlus schema, String viewSql, + List schemaPath, Boolean modifiable) { + return new ViewTableMacro(CalciteSchema.from(schema), viewSql, schemaPath, + modifiable); + } + + /** Returns the view's SQL definition. */ + public String getViewSql() { + return viewSql; + } + + /** Returns the the schema path of the view. */ + public List getSchemaPath() { + return schemaPath; } @Override public Schema.TableType getJdbcTableType() { @@ -80,8 +110,7 @@ public class ViewTable public Queryable asQueryable(QueryProvider queryProvider, SchemaPlus schema, String tableName) { return queryProvider.createQuery( - getExpression(schema, tableName, Queryable.class), - elementType); + getExpression(schema, tableName, Queryable.class), elementType); } public RelNode toRel( @@ -111,14 +140,16 @@ public class ViewTable static class ViewTableMacro implements TableMacro { protected final String viewSql; protected final CalciteSchema schema; + private final Boolean modifiable; /** Typically null. If specified, overrides the path of the schema as the * context for validating {@code viewSql}. */ protected final List schemaPath; - ViewTableMacro(CalciteSchema schema, String viewSql, - List schemaPath) { + ViewTableMacro(CalciteSchema schema, String viewSql, List schemaPath, + Boolean modifiable) { this.viewSql = viewSql; this.schema = schema; + this.modifiable = modifiable; this.schemaPath = schemaPath == null ? null : ImmutableList.copyOf(schemaPath); } @@ -128,25 +159,60 @@ public class ViewTable } public TranslatableTable apply(List arguments) { - CalcitePrepare.ParseResult parsed = - Schemas.parse(MaterializedViewTable.MATERIALIZATION_CONNECTION, - schema, schemaPath, viewSql); + CalcitePrepare.AnalyzeViewResult parsed = + Schemas.analyzeView(MaterializedViewTable.MATERIALIZATION_CONNECTION, + schema, schemaPath, viewSql, modifiable != null && modifiable); final List schemaPath1 = schemaPath != null ? schemaPath : schema.path(null); final JavaTypeFactory typeFactory = (JavaTypeFactory) parsed.typeFactory; - return new ViewTable(typeFactory.getJavaClass(parsed.rowType), - RelDataTypeImpl.proto(parsed.rowType), viewSql, schemaPath1); + final Type elementType = typeFactory.getJavaClass(parsed.rowType); + if ((modifiable == null || modifiable) && parsed.table != null) { + return new ModifiableViewTable(elementType, + RelDataTypeImpl.proto(parsed.rowType), viewSql, schemaPath1, + parsed.table, Schemas.path(schema.root(), parsed.tablePath), + parsed.constraint, parsed.columnMapping); + } else { + return new ViewTable(elementType, + RelDataTypeImpl.proto(parsed.rowType), viewSql, schemaPath1); + } } } - /** Returns the view's SQL definition. */ - public String getViewSql() { - return viewSql; - } + /** Extension to {@link ViewTable} that is modifiable. */ + static class ModifiableViewTable extends ViewTable + implements ModifiableView { + private final Table table; + private final Path tablePath; + private final RexNode constraint; + private final ImmutableIntList columnMapping; + + public ModifiableViewTable(Type elementType, RelProtoDataType rowType, + String viewSql, List schemaPath, Table table, + Path tablePath, RexNode constraint, + ImmutableIntList columnMapping) { + super(elementType, rowType, viewSql, schemaPath); + this.table = table; + this.tablePath = tablePath; + this.constraint = constraint; + this.columnMapping = columnMapping; + } - /** Returns the the schema path of the view. */ - public List getSchemaPath() { - return schemaPath; + public RexNode getConstraint(RexBuilder rexBuilder, + RelDataType tableRowType) { + return rexBuilder.copy(constraint); + } + + public ImmutableIntList getColumnMapping() { + return columnMapping; + } + + public Table getTable() { + return table; + } + + public Path getTablePath() { + return tablePath; + } } } http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java index 5dad7f4..a190159 100644 --- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java +++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java @@ -73,6 +73,8 @@ import org.apache.calcite.rex.RexUtil; import org.apache.calcite.rex.RexVisitorImpl; import org.apache.calcite.rex.RexWindowBound; import org.apache.calcite.schema.ModifiableTable; +import org.apache.calcite.schema.ModifiableView; +import org.apache.calcite.schema.Table; import org.apache.calcite.schema.TranslatableTable; import org.apache.calcite.sql.JoinConditionType; import org.apache.calcite.sql.JoinType; @@ -561,11 +563,7 @@ public class SqlToRelConverter { validatedRowType = uniquifyFields(validatedRowType); return RelOptUtil.equal( - "validated row type", - validatedRowType, - "converted row type", - convertedRowType, - false); + "validated row type", validatedRowType, "converted row type", convertedRowType, false); } protected RelDataType uniquifyFields(RelDataType rowType) { @@ -2024,7 +2022,8 @@ public class SqlToRelConverter { Set columnMappings = getColumnMappings(operator); LogicalTableFunctionScan callRel = - LogicalTableFunctionScan.create(cluster, + LogicalTableFunctionScan.create( + cluster, inputs, rexCall, elementType, @@ -2767,8 +2766,8 @@ public class SqlToRelConverter { protected RelNode createAggregate(Blackboard bb, boolean indicator, ImmutableBitSet groupSet, ImmutableList groupSets, List aggCalls) { - return LogicalAggregate.create(bb.root, indicator, groupSet, groupSets, - aggCalls); + return LogicalAggregate.create( + bb.root, indicator, groupSet, groupSets, aggCalls); } public RexDynamicParam convertDynamicParam( @@ -2999,27 +2998,88 @@ public class SqlToRelConverter { assert targetRowType != null; RelNode sourceRel = convertQueryRecursive( - call.getSource(), - false, - targetRowType); + call.getSource(), false, targetRowType); RelNode massagedRel = convertColumnList(call, sourceRel); + return createModify(targetTable, massagedRel); + } + + /** Creates a relational expression to modify a table or modifiable view. */ + private RelNode createModify(RelOptTable targetTable, RelNode source) { final ModifiableTable modifiableTable = targetTable.unwrap(ModifiableTable.class); if (modifiableTable != null) { - return modifiableTable.toModificationRel( - cluster, - targetTable, - catalogReader, - massagedRel, - LogicalTableModify.Operation.INSERT, - null, + return modifiableTable.toModificationRel(cluster, targetTable, + catalogReader, source, LogicalTableModify.Operation.INSERT, null, false); } - return LogicalTableModify.create(targetTable, catalogReader, massagedRel, + final ModifiableView modifiableView = + targetTable.unwrap(ModifiableView.class); + if (modifiableView != null) { + final Table delegateTable = modifiableView.getTable(); + final RelDataType delegateRowType = delegateTable.getRowType(typeFactory); + final RelOptTable delegateRelOptTable = + RelOptTableImpl.create(null, delegateRowType, delegateTable, + modifiableView.getTablePath()); + final RelNode newSource = + createSource(targetTable, source, modifiableView, delegateRowType); + return createModify(delegateRelOptTable, newSource); + } + return LogicalTableModify.create(targetTable, catalogReader, source, LogicalTableModify.Operation.INSERT, null, false); } + /** Wraps a relational expression in the projects and filters implied by + * a {@link ModifiableView}. + * + *

The input relational expression is suitable for inserting into the view, + * and the returned relational expression is suitable for inserting into its + * delegate table. + * + *

In principle, the delegate table of a view might be another modifiable + * view, and if so, the process can be repeated. */ + private RelNode createSource(RelOptTable targetTable, RelNode source, + ModifiableView modifiableView, RelDataType delegateRowType) { + final ImmutableIntList mapping = modifiableView.getColumnMapping(); + assert mapping.size() == targetTable.getRowType().getFieldCount(); + + // For columns represented in the mapping, the expression is just a field + // reference. + final Map projectMap = new HashMap<>(); + final List filters = new ArrayList<>(); + for (int i = 0; i < mapping.size(); i++) { + int target = mapping.get(i); + if (target >= 0) { + projectMap.put(target, RexInputRef.of(i, source.getRowType())); + } + } + + // For columns that are not in the mapping, and have a constraint of the + // form "column = value", the expression is the literal "value". + // + // If a column has multiple constraints, the extra ones will become a + // filter. + final RexNode constraint = + modifiableView.getConstraint(rexBuilder, delegateRowType); + RelOptUtil.inferViewPredicates(projectMap, filters, constraint); + final List> projects = new ArrayList<>(); + for (RelDataTypeField field : delegateRowType.getFieldList()) { + RexNode node = projectMap.get(field.getIndex()); + if (node == null) { + node = rexBuilder.makeNullLiteral(field.getType().getSqlTypeName()); + } + projects.add( + Pair.of(rexBuilder.ensureType(field.getType(), node, false), + field.getName())); + } + + source = RelOptUtil.createProject(source, projects, true); + if (filters.size() > 0) { + source = RelOptUtil.createFilter(source, filters); + } + return source; + } + private RelOptTable.ToRelContext createToRelContext() { return new RelOptTable.ToRelContext() { public RelOptCluster getCluster() { http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties ---------------------------------------------------------------------- diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties index e2321bb..e170b67 100644 --- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties +++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties @@ -187,4 +187,7 @@ StreamMustGroupByMonotonic=Streaming aggregation requires at least one monotonic StreamMustOrderByMonotonic=Streaming ORDER BY must start with monotonic expression StreamSetOpInconsistentInputs=Set operator cannot combine streaming and non-streaming inputs CannotStreamValues=Cannot stream VALUES +ModifiableViewMustBeBasedOnSingleTable=Modifiable view must be based on a single table +MoreThanOneMappedColumn=View is not modifiable. More than one expression maps to column ''{0}'' of base table ''{1}'' +NoValueSuppliedForViewColumn=View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}'' # End CalciteResource.properties http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java index 3694094..1a71ec4 100644 --- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java +++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java @@ -66,6 +66,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase { protected static final List SALES_TABLES = Arrays.asList( "TABLE(CATALOG.SALES.EMP)", + "TABLE(CATALOG.SALES.EMP_20)", "TABLE(CATALOG.SALES.EMP_ADDRESS)", "TABLE(CATALOG.SALES.DEPT)", "TABLE(CATALOG.SALES.BONUS)", http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/CalciteAssert.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java index cc4e1f2..8160c71 100644 --- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java +++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java @@ -698,7 +698,7 @@ public class CalciteAssert { + " ('Grace', 60, 'F'),\n" + " ('Wilma', cast(null as integer), 'F'))\n" + " as t(ename, deptno, gender)", - ImmutableList.of())); + ImmutableList.of(), null)); post.add("DEPT", ViewTable.viewMacro(post, "select * from (values\n" @@ -706,7 +706,7 @@ public class CalciteAssert { + " (20, 'Marketing'),\n" + " (30, 'Engineering'),\n" + " (40, 'Empty')) as t(deptno, dname)", - ImmutableList.of())); + ImmutableList.of(), null)); post.add("EMPS", ViewTable.viewMacro(post, "select * from (values\n" @@ -716,7 +716,7 @@ public class CalciteAssert { + " (120, 'Wilma', 20, 'F', CAST(NULL AS VARCHAR(20)), 1, 5, UNKNOWN, TRUE, DATE '2005-09-07'),\n" + " (130, 'Alice', 40, 'F', 'Vancouver', 2, CAST(NULL AS INT), FALSE, TRUE, DATE '2007-01-01'))\n" + " as t(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)", - ImmutableList.of())); + ImmutableList.of(), null)); return post; default: throw new AssertionError("unknown schema " + schema); http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java index a6d23e0..e4131c3 100644 --- a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java +++ b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java @@ -237,54 +237,59 @@ public class JdbcFrontLinqBackTest { employees.add(new JdbcTest.Employee(0, 0, "first", 0f, null)); return that() .with(CalciteAssert.Config.REGULAR) - .with(new CalciteAssert.ConnectionPostProcessor() { - public Connection apply(final Connection connection) - throws SQLException { - CalciteConnection calciteConnection = - connection.unwrap(CalciteConnection.class); - SchemaPlus rootSchema = - calciteConnection.getRootSchema(); - SchemaPlus mapSchema = - rootSchema.add("foo", new AbstractSchema()); - final String tableName = "bar"; - final JdbcTest.AbstractModifiableTable table = - new JdbcTest.AbstractModifiableTable(tableName) { - public RelDataType getRowType( - RelDataTypeFactory typeFactory) { - return ((JavaTypeFactory) typeFactory) - .createType(JdbcTest.Employee.class); - } + .with( + new CalciteAssert.ConnectionPostProcessor() { + public Connection apply(final Connection connection) + throws SQLException { + CalciteConnection calciteConnection = + connection.unwrap(CalciteConnection.class); + SchemaPlus rootSchema = + calciteConnection.getRootSchema(); + SchemaPlus mapSchema = + rootSchema.add("foo", new AbstractSchema()); + final String tableName = "bar"; + final JdbcTest.AbstractModifiableTable table = + mutable(tableName, employees); + mapSchema.add(tableName, table); + return calciteConnection; + } + }); + } - public Queryable asQueryable( - QueryProvider queryProvider, SchemaPlus schema, - String tableName) { - return new AbstractTableQueryable(queryProvider, - schema, this, tableName) { - public Enumerator enumerator() { - //noinspection unchecked - return (Enumerator) Linq4j.enumerator(employees); - } - }; - } + static JdbcTest.AbstractModifiableTable mutable(String tableName, + final List employees) { + return new JdbcTest.AbstractModifiableTable(tableName) { + public RelDataType getRowType( + RelDataTypeFactory typeFactory) { + return ((JavaTypeFactory) typeFactory) + .createType(JdbcTest.Employee.class); + } - public Type getElementType() { - return JdbcTest.Employee.class; - } + public Queryable asQueryable(QueryProvider queryProvider, + SchemaPlus schema, String tableName) { + return new AbstractTableQueryable(queryProvider, schema, this, + tableName) { + public Enumerator enumerator() { + //noinspection unchecked + return (Enumerator) Linq4j.enumerator(employees); + } + }; + } - public Expression getExpression(SchemaPlus schema, - String tableName, Class clazz) { - return Schemas.tableExpression(schema, getElementType(), - tableName, clazz); - } + public Type getElementType() { + return JdbcTest.Employee.class; + } - public Collection getModifiableCollection() { - return employees; - } - }; - mapSchema.add(tableName, table); - return calciteConnection; - } - }); + public Expression getExpression(SchemaPlus schema, String tableName, + Class clazz) { + return Schemas.tableExpression(schema, getElementType(), tableName, + clazz); + } + + public Collection getModifiableCollection() { + return employees; + } + }; } @Test public void testInsert2() { http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/JdbcTest.java ---------------------------------------------------------------------- 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 27eb0e5..8f73fbd 100644 --- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java +++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java @@ -58,6 +58,7 @@ import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.runtime.Hook; import org.apache.calcite.runtime.SqlFunctions; import org.apache.calcite.schema.ModifiableTable; +import org.apache.calcite.schema.ModifiableView; import org.apache.calcite.schema.QueryableTable; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaFactory; @@ -79,11 +80,13 @@ import org.apache.calcite.sql.advise.SqlAdvisorGetHintsFunction; import org.apache.calcite.sql.parser.SqlParserUtil; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.Bug; +import org.apache.calcite.util.JsonBuilder; import org.apache.calcite.util.Litmus; import org.apache.calcite.util.Pair; import org.apache.calcite.util.Util; import com.google.common.base.Function; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -130,6 +133,8 @@ import java.util.TimeZone; import java.util.regex.Pattern; import javax.sql.DataSource; +import static org.apache.calcite.util.Static.RESOURCE; + import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; @@ -137,6 +142,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.CoreMatchers.startsWith; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -250,6 +256,165 @@ public class JdbcTest { return FOODMART_QUERIES; } + /** Tests a modifiable view. */ + @Test public void testModelWithModifiableView() throws Exception { + final List employees = new ArrayList<>(); + employees.add(new Employee(135, 10, "Simon", 56.7f, null)); + try { + EmpDeptTableFactory.THREAD_COLLECTION.set(employees); + final CalciteAssert.AssertThat with = modelWithView( + "select \"name\", \"empid\" as e, \"salary\" " + + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" = 10", + null); + with.query("select \"name\" from \"adhoc\".V order by \"name\"") + .returns("name=Simon\n"); + with.doWithConnection( + new Function() { + @Override public Object apply(CalciteConnection input) { + try { + final Statement statement = input.createStatement(); + ResultSet resultSet = + statement.executeQuery("explain plan for\n" + + "insert into \"adhoc\".V\n" + + "values ('Fred', 56, 123.4)"); + assertThat(resultSet.next(), is(true)); + assertThat(resultSet.getString(1), + is( + "EnumerableTableModify(table=[[adhoc, MUTABLE_EMPLOYEES]], operation=[INSERT], updateColumnList=[[]], flattened=[false])\n" + + " EnumerableCalc(expr#0..2=[{inputs}], expr#3=[CAST($t1):JavaType(int) NOT NULL], expr#4=[10], expr#5=[CAST($t0):JavaType(class java.lang.String)], expr#6=[CAST($t2):JavaType(float) NOT NULL], expr#7=[null], empid=[$t3], deptno=[$t4], name=[$t5], salary=[$t6], commission=[$t7])\n" + + " EnumerableValues(tuples=[[{ 'Fred', 56, 123.4 }]])\n")); + + // With named columns + resultSet = + statement.executeQuery("explain plan for\n" + + "insert into \"adhoc\".V (\"name\", e, \"salary\")\n" + + "values ('Fred', 56, 123.4)"); + assertThat(resultSet.next(), is(true)); + + // With named columns, in different order + resultSet = + statement.executeQuery("explain plan for\n" + + "insert into \"adhoc\".V (e, \"salary\", \"name\")\n" + + "values (56, 123.4, 'Fred')"); + assertThat(resultSet.next(), is(true)); + + // Mis-named column + try { + final PreparedStatement s = + input.prepareStatement("explain plan for\n" + + "insert into \"adhoc\".V (empno, \"salary\", \"name\")\n" + + "values (56, 123.4, 'Fred')"); + fail("expected error, got " + s); + } catch (SQLException e) { + assertThat(e.getMessage(), + startsWith("Error while preparing statement")); + } + + // Fail to provide mandatory column + try { + final PreparedStatement s = + input.prepareStatement("explain plan for\n" + + "insert into \"adhoc\".V (e, name)\n" + + "values (56, 'Fred')"); + fail("expected error, got " + s); + } catch (SQLException e) { + assertThat(e.getMessage(), + startsWith("Error while preparing statement")); + } + + statement.close(); + return null; + } catch (SQLException e) { + throw Throwables.propagate(e); + } + } + }); + } finally { + EmpDeptTableFactory.THREAD_COLLECTION.remove(); + } + } + + /** Tests a few cases where modifiable views are invalid. */ + @Test public void testModelWithInvalidModifiableView() throws Exception { + final List employees = new ArrayList<>(); + employees.add(new Employee(135, 10, "Simon", 56.7f, null)); + try { + EmpDeptTableFactory.THREAD_COLLECTION.set(employees); + + Util.discard(RESOURCE.noValueSuppliedForViewColumn(null, null)); + modelWithView("select \"name\", \"empid\" as e, \"salary\" " + + "from \"MUTABLE_EMPLOYEES\" where \"commission\" = 10", + true) + .query("select \"name\" from \"adhoc\".V order by \"name\"") + .throws_( + "View is not modifiable. No value is supplied for NOT NULL " + + "column 'deptno' of base table 'MUTABLE_EMPLOYEES'"); + + // no error if we do not claim that the view is modifiable + modelWithView( + "select \"name\", \"empid\" as e, \"salary\" " + + "from \"MUTABLE_EMPLOYEES\" where \"commission\" = 10", null) + .query("select \"name\" from \"adhoc\".V order by \"name\"") + .runs(); + + modelWithView("select \"name\", \"empid\" as e, \"salary\" " + + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" IN (10, 20)", + true) + .query("select \"name\" from \"adhoc\".V order by \"name\"") + .throws_( + "View is not modifiable. No value is supplied for NOT NULL " + + "column 'deptno' of base table 'MUTABLE_EMPLOYEES'"); + + // Deduce "deptno = 10" from the constraint, and add a further + // condition "deptno < 20 OR commission > 1000". + modelWithView("select \"name\", \"empid\" as e, \"salary\" " + + "from \"MUTABLE_EMPLOYEES\"\n" + + "where \"deptno\" = 10 AND (\"deptno\" < 20 OR \"commission\" > 1000)", + true) + .query("insert into \"adhoc\".v values ('n',1,2)") + .explainContains("" + + "EnumerableTableModify(table=[[adhoc, MUTABLE_EMPLOYEES]], operation=[INSERT], updateColumnList=[[]], flattened=[false])\n" + + " EnumerableCalc(expr#0..2=[{inputs}], expr#3=[CAST($t1):JavaType(int) NOT NULL], expr#4=[10], expr#5=[CAST($t0):JavaType(class java.lang.String)], expr#6=[CAST($t2):JavaType(float) NOT NULL], expr#7=[null], expr#8=[20], expr#9=[<($t4, $t8)], expr#10=[1000], expr#11=[>($t7, $t10)], expr#12=[OR($t9, $t11)], empid=[$t3], deptno=[$t4], name=[$t5], salary=[$t6], commission=[$t7], $condition=[$t12])\n" + + " EnumerableValues(tuples=[[{ 'n', 1, 2 }]])"); + + modelWithView( + "select \"name\", \"empid\" as e, \"salary\" " + + "from \"MUTABLE_EMPLOYEES\"\n" + + "where \"commission\" = 100 AND \"deptno\" = 20", + true) + .query("select \"name\" from \"adhoc\".V order by \"name\"") + .runs(); + + modelWithView( + "select \"name\", \"empid\" as e, \"salary\", \"empid\" + 3 as e3, 1 as one\n" + + "from \"MUTABLE_EMPLOYEES\"\n" + + "where \"commission\" = 100 AND \"deptno\" = 20", + true) + .query("select \"name\" from \"adhoc\".V order by \"name\"") + .runs(); + + Util.discard(RESOURCE.moreThanOneMappedColumn(null, null)); + modelWithView( + "select \"name\", \"empid\" as e, \"salary\", \"name\" as n2 " + + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" IN (10, 20)", + true) + .query("select \"name\" from \"adhoc\".V order by \"name\"") + .throws_( + "View is not modifiable. More than one expression maps to " + + "column 'name' of base table 'MUTABLE_EMPLOYEES'"); + + // no error if we do not claim that the view is modifiable + modelWithView( + "select \"name\", \"empid\" as e, \"salary\", \"name\" as n2 " + + "from \"MUTABLE_EMPLOYEES\" where \"deptno\" IN (10, 20)", + null) + .query("select \"name\" from \"adhoc\".V order by \"name\"") + .runs(); + } finally { + EmpDeptTableFactory.THREAD_COLLECTION.remove(); + } + } + /** * Tests a table function with literal arguments. */ @@ -3264,10 +3429,10 @@ public class JdbcTest { + " EnumerableCalc(expr#0..4=[{inputs}], expr#5=[+($t3, $t0)], proj#0..1=[{exprs}], salary=[$t3], $3=[$t5])\n" + " EnumerableTableScan(table=[[hr, emps]])\n") .returnsUnordered( - "deptno=10; empid=100; S=10100.0; FIVE=5; M=10000.0; C=1" - , "deptno=10; empid=110; S=21710.0; FIVE=5; M=10000.0; C=2" - , "deptno=10; empid=150; S=18760.0; FIVE=5; M=7000.0; C=2" - , "deptno=20; empid=200; S=8200.0; FIVE=5; M=8000.0; C=1") + "deptno=10; empid=100; S=10100.0; FIVE=5; M=10000.0; C=1", + "deptno=10; empid=110; S=21710.0; FIVE=5; M=10000.0; C=2", + "deptno=10; empid=150; S=18760.0; FIVE=5; M=7000.0; C=2", + "deptno=20; empid=200; S=8200.0; FIVE=5; M=8000.0; C=1") .planContains(CalcitePrepareImpl.DEBUG ? "_list.add(new Object[] {\n" + " row[0],\n" // box-unbox is optimized @@ -4822,7 +4987,8 @@ public class JdbcTest { "Cannot define view; parent schema 'adhoc' is not mutable"); } - private CalciteAssert.AssertThat modelWithView(String view) { + private CalciteAssert.AssertThat modelWithView(String view, + Boolean modifiable) { final Class clazz = EmpDeptTableFactory.class; return CalciteAssert.model("{\n" + " version: '1.0',\n" @@ -4837,9 +5003,16 @@ public class JdbcTest { + " operand: {'foo': true, 'bar': 345}\n" + " },\n" + " {\n" + + " name: 'MUTABLE_EMPLOYEES',\n" + + " type: 'custom',\n" + + " factory: '" + clazz.getName() + "',\n" + + " operand: {'foo': false}\n" + + " },\n" + + " {\n" + " name: 'V',\n" + " type: 'view',\n" - + " sql: '" + view + "'\n" + + (modifiable == null ? "" : " modifiable: " + modifiable + ",\n") + + " sql: " + new JsonBuilder().toJsonString(view) + "\n" + " }\n" + " ]\n" + " }\n" @@ -4850,7 +5023,8 @@ public class JdbcTest { /** Tests a JDBC connection that provides a model that contains a view. */ @Test public void testModelView() throws Exception { final CalciteAssert.AssertThat with = - modelWithView("select * from \"EMPLOYEES\" where \"deptno\" = 10"); + modelWithView("select * from \"EMPLOYEES\" where \"deptno\" = 10", + null); with.query("select * from \"adhoc\".V order by \"name\" desc") .returns("" @@ -4868,6 +5042,7 @@ public class JdbcTest { // all table types assertEquals( "TABLE_CAT=null; TABLE_SCHEM=adhoc; TABLE_NAME=EMPLOYEES; TABLE_TYPE=TABLE; REMARKS=null; TYPE_CAT=null; TYPE_SCHEM=null; TYPE_NAME=null; SELF_REFERENCING_COL_NAME=null; REF_GENERATION=null\n" + + "TABLE_CAT=null; TABLE_SCHEM=adhoc; TABLE_NAME=MUTABLE_EMPLOYEES; TABLE_TYPE=TABLE; REMARKS=null; TYPE_CAT=null; TYPE_SCHEM=null; TYPE_NAME=null; SELF_REFERENCING_COL_NAME=null; REF_GENERATION=null\n" + "TABLE_CAT=null; TABLE_SCHEM=adhoc; TABLE_NAME=V; TABLE_TYPE=VIEW; REMARKS=null; TYPE_CAT=null; TYPE_SCHEM=null; TYPE_NAME=null; SELF_REFERENCING_COL_NAME=null; REF_GENERATION=null\n", CalciteAssert.toString( metaData.getTables(null, "adhoc", null, null))); @@ -4931,7 +5106,7 @@ public class JdbcTest { @Test public void testOrderByView() throws Exception { final CalciteAssert.AssertThat with = modelWithView("select * from \"EMPLOYEES\" where \"deptno\" = 10 " - + "order by \"empid\" limit 2"); + + "order by \"empid\" limit 2", null); with .query("select \"name\" from \"adhoc\".V order by \"name\"") .returns("name=Bill\n" @@ -6050,7 +6225,9 @@ public class JdbcTest { assertThat(a2CalciteSchema.getTable("table1", false), notNullValue()); assertThat(a2CalciteSchema.getTable("taBle1", true), nullValue()); assertThat(a2CalciteSchema.getTable("taBle1", false), notNullValue()); - final TableMacro function = ViewTable.viewMacro(a2Schema, "values 1", null); + final TableMacro function = + ViewTable.viewMacro(a2Schema, "values 1", null, null); + Util.discard(function); connection.close(); } @@ -6186,7 +6363,7 @@ public class JdbcTest { assertThat(rs.next(), is(true)); assertThat((Integer) rs.getObject("ID"), equalTo(2)); - assertThat((Double) rs.getObject("VALS"), nullValue()); + assertThat(rs.getObject("VALS"), nullValue()); assertThat(rs.next(), is(true)); assertThat(rs.getObject("ID"), nullValue()); @@ -6420,8 +6597,19 @@ public class JdbcTest { } } + /** Abstract base class for implementations of {@link ModifiableView}. */ + public abstract static class AbstractModifiableView + extends AbstractTable implements ModifiableView { + protected AbstractModifiableView() { + super(); + } + } + /** Factory for EMP and DEPT tables. */ public static class EmpDeptTableFactory implements TableFactory { + public static final ThreadLocal> THREAD_COLLECTION = + new ThreadLocal<>(); + public Table create( SchemaPlus schema, String name, @@ -6429,12 +6617,23 @@ public class JdbcTest { RelDataType rowType) { final Class clazz; final Object[] array; - if (name.equals("EMPLOYEES")) { + switch (name) { + case "EMPLOYEES": clazz = Employee.class; array = new HrSchema().emps; - } else { + break; + case "MUTABLE_EMPLOYEES": + List employees = THREAD_COLLECTION.get(); + if (employees == null) { + employees = Collections.emptyList(); + } + return JdbcFrontLinqBackTest.mutable(name, employees); + case "DEPARTMENTS": clazz = Department.class; array = new HrSchema().depts; + break; + default: + throw new AssertionError(name); } return new AbstractQueryableTable(clazz) { public RelDataType getRowType(RelDataTypeFactory typeFactory) {