You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2015/08/01 00:43:12 UTC

[35/50] [abbrv] incubator-calcite git commit: [CALCITE-505] Support modifiable view

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
index 6e7fdf3..36d7788 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptSchema;
@@ -27,14 +29,27 @@ import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeComparability;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+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.sql.SqlAccessType;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.ObjectSqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -46,26 +61,31 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 
-import java.util.ArrayList;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.AbstractList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 
 /**
  * Mock implementation of {@link SqlValidatorCatalogReader} which returns tables
  * "EMP", "DEPT", "BONUS", "SALGRADE" (same as Oracle's SCOTT schema).
+ * Also two streams "ORDERS", "SHIPMENTS";
+ * and a view "EMP_20".
  */
 public class MockCatalogReader implements Prepare.CatalogReader {
   //~ Static fields/initializers ---------------------------------------------
@@ -100,12 +120,11 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     this.typeFactory = typeFactory;
     this.caseSensitive = caseSensitive;
     if (caseSensitive) {
-      tables = new HashMap<List<String>, MockTable>();
-      schemas = new HashMap<String, MockSchema>();
+      tables = Maps.newHashMap();
+      schemas = Maps.newHashMap();
     } else {
-      tables = new TreeMap<List<String>, MockTable>(
-          CASE_INSENSITIVE_LIST_COMPARATOR);
-      schemas = new TreeMap<String, MockSchema>(String.CASE_INSENSITIVE_ORDER);
+      tables = Maps.newTreeMap(CASE_INSENSITIVE_LIST_COMPARATOR);
+      schemas = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
     }
   }
 
@@ -147,7 +166,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     registerSchema(salesSchema);
 
     // Register "EMP" table.
-    MockTable empTable = MockTable.create(this, salesSchema, "EMP", false);
+    final MockTable empTable =
+        MockTable.create(this, salesSchema, "EMP", false);
     empTable.addColumn("EMPNO", intType);
     empTable.addColumn("ENAME", varchar20Type);
     empTable.addColumn("JOB", varchar10Type);
@@ -228,6 +248,122 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     shipmentsStream.addColumn("ORDERID", intType);
     registerTable(shipmentsStream);
 
+    // Register "EMP_20" view.
+    // Same columns as "EMP",
+    // but "DEPTNO" not visible and set to 20 by default
+    // and "SAL" is visible but must be greater than 1000
+    MockTable emp20View = new MockTable(this, salesSchema.getCatalogName(),
+        salesSchema.name, "EMP_20", false) {
+      private final Table table = empTable.unwrap(Table.class);
+      private final ImmutableIntList mapping =
+          ImmutableIntList.of(0, 1, 2, 3, 4, 5, 6, 8);
+
+      @Override public RelNode toRel(ToRelContext context) {
+        // Expand to the equivalent of:
+        //   SELECT EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, SLACKER
+        //   FROM EMP
+        //   WHERE DEPTNO = 20 AND SAL > 1000
+        RelNode rel = LogicalTableScan.create(context.getCluster(), empTable);
+        final RexBuilder rexBuilder = context.getCluster().getRexBuilder();
+        rel = LogicalFilter.create(rel,
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.AND,
+                rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                    rexBuilder.makeInputRef(rel, 7),
+                    rexBuilder.makeExactLiteral(BigDecimal.valueOf(20))),
+                rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN,
+                    rexBuilder.makeInputRef(rel, 5),
+                    rexBuilder.makeExactLiteral(BigDecimal.valueOf(1000)))));
+        final List<RelDataTypeField> fieldList =
+            rel.getRowType().getFieldList();
+        final List<Pair<RexNode, String>> projects =
+            new AbstractList<Pair<RexNode, String>>() {
+              @Override public Pair<RexNode, String> get(int index) {
+                return RexInputRef.of2(mapping.get(index), fieldList);
+              }
+
+              @Override public int size() {
+                return mapping.size();
+              }
+            };
+        return LogicalProject.create(rel, Pair.left(projects),
+            Pair.right(projects));
+      }
+
+      @Override public <T> T unwrap(Class<T> clazz) {
+        if (clazz.isAssignableFrom(ModifiableView.class)) {
+          return clazz.cast(
+              new JdbcTest.AbstractModifiableView() {
+                @Override public Table getTable() {
+                  return empTable.unwrap(Table.class);
+                }
+
+                @Override public Path getTablePath() {
+                  final ImmutableList.Builder<Pair<String, Schema>> builder =
+                      ImmutableList.builder();
+                  builder.add(Pair.<String, Schema>of(empTable.names.get(0), null));
+                  builder.add(Pair.<String, Schema>of(empTable.names.get(1), null));
+                  builder.add(Pair.<String, Schema>of(empTable.names.get(2), null));
+                  return Schemas.path(builder.build());
+//                  return empTable.names;
+                }
+
+                @Override public ImmutableIntList getColumnMapping() {
+                  return mapping;
+                }
+
+                @Override public RexNode getConstraint(RexBuilder rexBuilder,
+                    RelDataType tableRowType) {
+                  final RelDataTypeField deptnoField =
+                      tableRowType.getFieldList().get(7);
+                  final RelDataTypeField salField =
+                      tableRowType.getFieldList().get(5);
+                  final List<RexNode> nodes = Arrays.asList(
+                      rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                          rexBuilder.makeInputRef(deptnoField.getType(),
+                              deptnoField.getIndex()),
+                          rexBuilder.makeExactLiteral(BigDecimal.valueOf(20L),
+                              deptnoField.getType())),
+                      rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN,
+                          rexBuilder.makeInputRef(salField.getType(),
+                              salField.getIndex()),
+                          rexBuilder.makeExactLiteral(BigDecimal.valueOf(1000L),
+                              salField.getType())));
+                  return RexUtil.composeConjunction(rexBuilder, nodes, false);
+                }
+
+                @Override public RelDataType
+                getRowType(final RelDataTypeFactory typeFactory) {
+                  return typeFactory.createStructType(
+                      new AbstractList<Map.Entry<String, RelDataType>>() {
+                        @Override public Map.Entry<String, RelDataType>
+                        get(int index) {
+                          return table.getRowType(typeFactory).getFieldList()
+                              .get(mapping.get(index));
+                        }
+
+                        @Override public int size() {
+                          return mapping.size();
+                        }
+                      }
+                  );
+                }
+              });
+        }
+        return super.unwrap(clazz);
+      }
+    };
+    salesSchema.addTable(Util.last(emp20View.getQualifiedName()));
+    emp20View.addColumn("EMPNO", intType);
+    emp20View.addColumn("ENAME", varchar20Type);
+    emp20View.addColumn("JOB", varchar10Type);
+    emp20View.addColumn("MGR", intTypeNull);
+    emp20View.addColumn("HIREDATE", timestampType);
+    emp20View.addColumn("SAL", intType);
+    emp20View.addColumn("COMM", intType);
+    emp20View.addColumn("SLACKER", booleanType);
+    registerTable(emp20View);
+
     return this;
   }
 
@@ -296,7 +432,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
           .build();
     case 1:
       // looking for schema names
-      result = new ArrayList<SqlMoniker>();
+      result = Lists.newArrayList();
       for (MockSchema schema : schemas.values()) {
         final String catalogName = names.get(0);
         if (schema.getCatalogName().equals(catalogName)) {
@@ -312,7 +448,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       if (schema == null) {
         return Collections.emptyList();
       }
-      result = new ArrayList<SqlMoniker>();
+      result = Lists.newArrayList();
       for (String tableName : schema.tableNames) {
         result.add(
             new SqlMonikerImpl(
@@ -330,30 +466,6 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     return ImmutableList.of(DEFAULT_CATALOG, DEFAULT_SCHEMA);
   }
 
-  private MockSchema getMockSchema(List<String> names) {
-    return schemas.get(names.get(0));
-  }
-
-  public List<SqlMoniker> getAllSchemaObjectNames2(List<String> names) {
-    List<SqlMoniker> result = new ArrayList<SqlMoniker>();
-    if (names.isEmpty()) {
-      for (MockSchema schema : schemas.values()) {
-        result.add(
-            new SqlMonikerImpl(schema.name, SqlMonikerType.SCHEMA));
-      }
-    }
-    // looking for table names in the given schema
-    MockSchema schema = getMockSchema(names);
-    if (schema != null) {
-      for (String tableName : schema.tableNames) {
-        result.add(
-            new SqlMonikerImpl(
-                tableName, SqlMonikerType.TABLE));
-      }
-    }
-    return result;
-  }
-
   public RelDataTypeField field(RelDataType rowType, String alias) {
     return SqlValidatorUtil.lookupField(caseSensitive, elideRecord, rowType,
         alias);
@@ -380,8 +492,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   private static List<RelCollation> deduceMonotonicity(
       Prepare.PreparingTable table) {
-    final List<RelCollation> collationList =
-        new ArrayList<RelCollation>();
+    final List<RelCollation> collationList = Lists.newArrayList();
 
     // Deduce which fields the table is sorted on.
     int i = -1;
@@ -407,7 +518,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   /** Mock schema. */
   public static class MockSchema {
-    private final List<String> tableNames = new ArrayList<String>();
+    private final List<String> tableNames = Lists.newArrayList();
     private String name;
 
     public MockSchema(String name) {
@@ -428,13 +539,13 @@ public class MockCatalogReader implements Prepare.CatalogReader {
    * {@link org.apache.calcite.prepare.Prepare.PreparingTable}.
    */
   public static class MockTable implements Prepare.PreparingTable {
-    private final MockCatalogReader catalogReader;
+    protected final MockCatalogReader catalogReader;
     private final boolean stream;
     private final List<Map.Entry<String, RelDataType>> columnList =
         Lists.newArrayList();
     private RelDataType rowType;
     private List<RelCollation> collationList;
-    private final List<String> names;
+    protected final List<String> names;
     private final Set<String> monotonicColumnSet = Sets.newHashSet();
 
     public MockTable(MockCatalogReader catalogReader, String catalogName,
@@ -457,6 +568,34 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       if (clazz.isInstance(this)) {
         return clazz.cast(this);
       }
+      if (clazz.isAssignableFrom(Table.class)) {
+        return clazz.cast(
+            new JdbcTest.AbstractModifiableTable(Util.last(names)) {
+              @Override public RelDataType
+              getRowType(RelDataTypeFactory typeFactory) {
+                return typeFactory.createStructType(rowType.getFieldList());
+              }
+
+              @Override public Collection getModifiableCollection() {
+                return null;
+              }
+
+              @Override public <T> Queryable<T>
+              asQueryable(QueryProvider queryProvider, SchemaPlus schema,
+                  String tableName) {
+                return null;
+              }
+
+              @Override public Type getElementType() {
+                return null;
+              }
+
+              @Override public Expression getExpression(SchemaPlus schema,
+                  String tableName, Class clazz) {
+                return null;
+              }
+            });
+      }
       return null;
     }
 
@@ -515,10 +654,6 @@ public class MockCatalogReader implements Prepare.CatalogReader {
       throw new UnsupportedOperationException();
     }
 
-    public void addColumn(int index, String name, RelDataType type) {
-      columnList.add(index, Pair.of(name, type));
-    }
-
     public void addColumn(String name, RelDataType type) {
       columnList.add(Pair.of(name, type));
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
index 7c5ba6f..aaec45d 100644
--- a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
@@ -198,7 +198,7 @@ public class ReflectiveSchemaTest {
     schema.add("emps_view",
         ViewTable.viewMacro(schema,
             "select * from \"hr\".\"emps\" where \"deptno\" = 10",
-            null));
+            null, null));
     rootSchema.add("hr", new ReflectiveSchema(new JdbcTest.HrSchema()));
     ResultSet resultSet = connection.createStatement().executeQuery(
         "select *\n"
@@ -224,17 +224,17 @@ public class ReflectiveSchemaTest {
     schema.add("emps",
         ViewTable.viewMacro(schema,
             "select * from \"emps\" where \"deptno\" = 10",
-            Collections.singletonList("hr")));
+            Collections.singletonList("hr"), null));
     schema.add("hr_emps",
         ViewTable.viewMacro(schema,
             "select * from \"emps\"",
-            Collections.singletonList("hr")));
+            Collections.singletonList("hr"), null));
     schema.add("s_emps",
         ViewTable.viewMacro(schema,
             "select * from \"emps\"",
-            Collections.singletonList("s")));
+            Collections.singletonList("s"), null));
     schema.add("null_emps",
-        ViewTable.viewMacro(schema, "select * from \"emps\"", null));
+        ViewTable.viewMacro(schema, "select * from \"emps\"", null, null));
     rootSchema.add("hr", new ReflectiveSchema(new JdbcTest.HrSchema()));
     final Statement statement = connection.createStatement();
     ResultSet resultSet;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 1853244..08a01a9 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1142,6 +1142,22 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
         .convertsTo("${plan}");
   }
 
+  @Test public void testInsert() {
+    sql("insert into emp (deptno, empno, ename) values (10, 150, 'Fred')")
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testSelectView() {
+    // translated condition: deptno = 20 and sal > 1000 and empno > 100
+    sql("select * from emp_20 where empno > 100")
+        .convertsTo("${plan}");
+  }
+
+  @Test public void testInsertView() {
+    sql("insert into emp_20 (empno, ename) values (150, 'Fred')")
+        .convertsTo("${plan}");
+  }
+
   /**
    * Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-695">[CALCITE-695]

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index a2e6c20..2bf64c7 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -7079,6 +7079,12 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     tester1.checkQuery("select empNo from (select Empno from emP)");
   }
 
+  @Test public void testInsert() {
+    tester.checkQuery("insert into emp (empno, deptno) values (1, 1)");
+    tester.checkQuery("insert into emp (empno, deptno)\n"
+        + "select 1, 1 from (values 'a')");
+  }
+
   @Test public void testStream() {
     sql("select stream * from orders").ok();
     sql("select stream * from ^emp^")

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index ef17e98..b3be8ce 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2578,4 +2578,44 @@ LogicalProject(DEPTNO=[$0], B=[>($1, $2)])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testInsert">
+        <Resource name="sql">
+            <![CDATA[insert into emp (deptno, empno, ename) values (10, 150, 'Fred')]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalTableModify(table=[[CATALOG, SALES, EMP]], operation=[INSERT], updateColumnList=[[]], flattened=[true])
+  LogicalProject(EMPNO=[$1], ENAME=[$2], JOB=[null], MGR=[null], HIREDATE=[null], SAL=[null], COMM=[null], DEPTNO=[$0], SLACKER=[null])
+    LogicalValues(tuples=[[{ 10, 150, 'Fred' }]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSelectView">
+        <Resource name="sql">
+            <![CDATA[select * from emp_20 where empno > 100]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$7])
+  LogicalFilter(condition=[>($0, 100)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$8])
+      LogicalFilter(condition=[AND(=($7, 20), >($5, 1000))])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testInsertView">
+        <Resource name="sql">
+            <![CDATA[insert into emp_20 (empno, ename) values (150, 'Fred')]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalTableModify(table=[[CATALOG, SALES, EMP]], operation=[INSERT], updateColumnList=[[]], flattened=[true])
+  LogicalFilter(condition=[>($5, 1000)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[CAST($2):VARCHAR(10) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL], MGR=[$3], HIREDATE=[CAST($4):TIMESTAMP(0) NOT NULL], SAL=[CAST($5):INTEGER NOT NULL], COMM=[CAST($6):INTEGER NOT NULL], DEPTNO=[20], SLACKER=[CAST($7):BOOLEAN NOT NULL])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[null], MGR=[null], HIREDATE=[null], SAL=[null], COMM=[null], SLACKER=[null])
+        LogicalValues(tuples=[[{ 150, 'Fred' }]])
+]]>
+        </Resource>
+    </TestCase>
 </Root>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/doc/model.md
----------------------------------------------------------------------
diff --git a/doc/model.md b/doc/model.md
index 1b2666d..3f49159 100644
--- a/doc/model.md
+++ b/doc/model.md
@@ -224,7 +224,8 @@ Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
 {
   name: 'female_emps',
   type: 'view',
-  sql: "select * from emps where gender = 'F'"
+  sql: "select * from emps where gender = 'F'",
+  modifiable: true
 }
 ```
 
@@ -236,6 +237,29 @@ Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
 `path` (optional list) is the SQL path to resolve the query. If not
 specified, defaults to the current schema.
 
+`modifiable` (optional boolean) is whether the view is modifiable.
+If null or not specified, Calcite deduces whether the view is modifiable.
+
+A view is modifiable if contains only SELECT, FROM, WHERE (no JOIN, aggregation
+or sub-queries) and every column:
+* is specified once in the SELECT clause; or
+* occurs in the WHERE clause with a `column = literal` predicate; or
+* is nullable.
+
+The second clause allows Calcite to automatically provide the correct value for
+hidden columns. It is useful in multi-tenant environments, where the `tenantId`
+column is hidden, mandatory (NOT NULL), and has a constant value for a
+particular view.
+
+Errors regarding modifiable views:
+* If a view is marked `modifiable: true` and is not modifiable, Calcite throws
+  an error while reading the schema.
+* If you submit an INSERT, UPDATE or UPSERT command to a non-modifiable view,
+  Calcite throws an error when validating the statement.
+* If a DML statement creates a row that would not appear in the view
+  (for example, a row in `female_emps`, above, with `gender = 'M'`),
+  Calcite throws an error when executing the statement.
+
 ### Custom Table
 
 Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
index c9a22de..54b49f1 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
@@ -43,7 +43,9 @@ public class ConstantExpression extends Expression {
         if (primitive != null) {
           clazz = primitive.boxClass;
         }
-        if (!clazz.isInstance(value)) {
+        if (!clazz.isInstance(value)
+            && !((clazz == Float.class || clazz == Double.class)
+                && value instanceof BigDecimal)) {
           throw new AssertionError(
               "value " + value + " does not match type " + type);
         }
@@ -85,6 +87,7 @@ public class ConstantExpression extends Expression {
       return writer;
     }
     final Primitive primitive = Primitive.of(type);
+    final BigDecimal bigDecimal;
     if (primitive != null) {
       switch (primitive) {
       case BYTE:
@@ -96,8 +99,28 @@ public class ConstantExpression extends Expression {
       case LONG:
         return writer.append(value).append("L");
       case FLOAT:
+        if (value instanceof BigDecimal) {
+          bigDecimal = (BigDecimal) value;
+        } else {
+          bigDecimal = BigDecimal.valueOf((Float) value);
+        }
+        if (bigDecimal.precision() > 6) {
+          return writer.append("Float.intBitsToFloat(")
+              .append(Float.floatToIntBits(bigDecimal.floatValue()))
+              .append(")");
+        }
         return writer.append(value).append("F");
       case DOUBLE:
+        if (value instanceof BigDecimal) {
+          bigDecimal = (BigDecimal) value;
+        } else {
+          bigDecimal = BigDecimal.valueOf((Double) value);
+        }
+        if (bigDecimal.precision() > 10) {
+          return writer.append("Double.longBitsToDouble(")
+              .append(Double.doubleToLongBits(bigDecimal.doubleValue()))
+              .append("L)");
+        }
         return writer.append(value).append("D");
       default:
         return writer.append(value);
@@ -115,7 +138,7 @@ public class ConstantExpression extends Expression {
           .append(((Enum) value).name());
     }
     if (value instanceof BigDecimal) {
-      BigDecimal bigDecimal = ((BigDecimal) value).stripTrailingZeros();
+      bigDecimal = ((BigDecimal) value).stripTrailingZeros();
       try {
         final int scale = bigDecimal.scale();
         final long exact = bigDecimal.scaleByPowerOfTen(scale).longValueExact();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
index 3936d5f..fae932e 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Expressions.java
@@ -566,7 +566,11 @@ public abstract class Expressions {
       if (primitive != null) {
         clazz = primitive.boxClass;
       }
-      if (!clazz.isInstance(value)) {
+      if ((clazz == Float.class || clazz == Double.class)
+          && value instanceof BigDecimal) {
+        // Don't try to convert the value of float and double literals.
+        // We'd experience rounding, e.g. 3.2 becomes 3.1999998.
+      } else if (!clazz.isInstance(value)) {
         String stringValue = String.valueOf(value);
         if (type == BigDecimal.class) {
           value = new BigDecimal(stringValue);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc45a2c0/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
index 7a4f65a..338251b 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/Linq4jTest.java
@@ -34,6 +34,7 @@ import org.apache.calcite.linq4j.function.Functions;
 import org.apache.calcite.linq4j.function.IntegerFunction1;
 import org.apache.calcite.linq4j.function.Predicate1;
 import org.apache.calcite.linq4j.function.Predicate2;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
 
@@ -43,6 +44,7 @@ import com.google.common.collect.Lists;
 
 import org.junit.Test;
 
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -1660,6 +1662,37 @@ public class Linq4jTest {
     Linq4jExample.main(new String[0]);
   }
 
+  /** We use BigDecimal to represent literals of float and double using
+   * BigDecimal, because we want an exact representation. */
+  @Test public void testApproxConstant() {
+    ConstantExpression c;
+    c = Expressions.constant(new BigDecimal("3.1"), float.class);
+    assertThat(Expressions.toString(c), equalTo("3.1F"));
+    c = Expressions.constant(new BigDecimal("-5.156"), float.class);
+    assertThat(Expressions.toString(c), equalTo("-5.156F"));
+    c = Expressions.constant(new BigDecimal("-51.6"), Float.class);
+    assertThat(Expressions.toString(c), equalTo("Float.valueOf(-51.6F)"));
+    c = Expressions.constant(new BigDecimal(Float.MAX_VALUE), Float.class);
+    assertThat(Expressions.toString(c),
+        equalTo("Float.valueOf(Float.intBitsToFloat(2139095039))"));
+    c = Expressions.constant(new BigDecimal(Float.MIN_VALUE), Float.class);
+    assertThat(Expressions.toString(c),
+        equalTo("Float.valueOf(Float.intBitsToFloat(1))"));
+
+    c = Expressions.constant(new BigDecimal("3.1"), double.class);
+    assertThat(Expressions.toString(c), equalTo("3.1D"));
+    c = Expressions.constant(new BigDecimal("-5.156"), double.class);
+    assertThat(Expressions.toString(c), equalTo("-5.156D"));
+    c = Expressions.constant(new BigDecimal("-51.6"), Double.class);
+    assertThat(Expressions.toString(c), equalTo("Double.valueOf(-51.6D)"));
+    c = Expressions.constant(new BigDecimal(Double.MAX_VALUE), Double.class);
+    assertThat(Expressions.toString(c),
+        equalTo("Double.valueOf(Double.longBitsToDouble(9218868437227405311L))"));
+    c = Expressions.constant(new BigDecimal(Double.MIN_VALUE), Double.class);
+    assertThat(Expressions.toString(c),
+        equalTo("Double.valueOf(Double.longBitsToDouble(1L))"));
+  }
+
   /** Employee. */
   public static class Employee {
     public final int empno;