You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by da...@apache.org on 2019/07/22 01:24:00 UTC

[calcite] branch master updated: [CALCITE-3201] SqlValidator throws exception for sql insert target table with virtual columns

This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/master by this push:
     new 9c11efb  [CALCITE-3201] SqlValidator throws exception for sql insert target table with virtual columns
9c11efb is described below

commit 9c11efb5aae075f473d7cbfefbac99af1ef31553
Author: yuzhao.cyz <yu...@alibaba-inc.com>
AuthorDate: Thu Jul 18 18:21:43 2019 +0800

    [CALCITE-3201] SqlValidator throws exception for sql insert target table with virtual columns
---
 .../org/apache/calcite/schema/ColumnStrategy.java  | 10 +++-
 .../calcite/sql/validate/SqlValidatorImpl.java     | 68 +++++++++++++++-------
 .../org/apache/calcite/test/SqlValidatorTest.java  | 28 +++++++++
 .../org/apache/calcite/test/catalog/Fixture.java   |  4 ++
 .../test/catalog/MockCatalogReaderExtended.java    | 24 ++++++++
 .../catalog/VirtualColumnsExpressionFactory.java}  | 38 ++++++------
 .../java/org/apache/calcite/test/ServerTest.java   |  5 +-
 7 files changed, 131 insertions(+), 46 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/schema/ColumnStrategy.java b/core/src/main/java/org/apache/calcite/schema/ColumnStrategy.java
index f51175d..1f633b8 100644
--- a/core/src/main/java/org/apache/calcite/schema/ColumnStrategy.java
+++ b/core/src/main/java/org/apache/calcite/schema/ColumnStrategy.java
@@ -36,7 +36,15 @@ public enum ColumnStrategy {
   /** Column is computed and stored. You cannot insert into it. */
   STORED,
   /** Column is computed and not stored. You cannot insert into it. */
-  VIRTUAL
+  VIRTUAL;
+
+  /**
+   * Returns whether you can insert into the column.
+   * @return true if this column can be inserted.
+   */
+  public boolean canInsertInto() {
+    return this != STORED && this != VIRTUAL;
+  }
 }
 
 // End ColumnStrategy.java
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index adfa42b..65aa98b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -29,7 +29,6 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelRecordType;
-import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPatternFieldRef;
 import org.apache.calcite.rex.RexVisitor;
@@ -90,7 +89,6 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
 import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.util.SqlVisitor;
-import org.apache.calcite.sql2rel.InitializerContext;
 import org.apache.calcite.util.BitString;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -131,6 +129,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.function.Supplier;
+import java.util.stream.Collectors;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
@@ -4261,14 +4260,34 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     final RelDataType logicalSourceRowType =
         getLogicalSourceRowType(sourceRowType, insert);
 
-    checkFieldCount(insert.getTargetTable(), table, source,
-        logicalSourceRowType, logicalTargetRowType);
+    final List<ColumnStrategy> strategies =
+        table.unwrap(RelOptTable.class).getColumnStrategies();
+
+    final RelDataType realTargetRowType = typeFactory.createStructType(
+        logicalTargetRowType.getFieldList()
+            .stream().filter(f -> strategies.get(f.getIndex()).canInsertInto())
+            .collect(Collectors.toList()));
+
+    final RelDataType targetRowTypeToValidate =
+        logicalSourceRowType.getFieldCount() == logicalTargetRowType.getFieldCount()
+        ? logicalTargetRowType
+        : realTargetRowType;
 
-    checkTypeAssignment(logicalSourceRowType, logicalTargetRowType, insert);
+    checkFieldCount(insert.getTargetTable(), table, strategies,
+        targetRowTypeToValidate, realTargetRowType,
+        source, logicalSourceRowType, logicalTargetRowType);
+
+    // Skip the virtual columns(can not insert into) type assignment
+    // check if the source fields num is equals with
+    // the real target table fields num, see how #checkFieldCount was used.
+    checkTypeAssignment(logicalSourceRowType, targetRowTypeToValidate, insert);
 
     checkConstraint(table, source, logicalTargetRowType);
 
     validateAccess(insert.getTargetTable(), table, SqlAccessEnum.INSERT);
+
+    // Refresh the insert row type to keep sync with source.
+    setValidatedNodeType(insert, targetRowTypeToValidate);
   }
 
   /**
@@ -4367,31 +4386,38 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
+  /**
+   * Check the field count of sql insert source and target node row type.
+   * @param node                    target table sql identifier
+   * @param table                   target table
+   * @param strategies              column strategies of target table
+   * @param targetRowTypeToValidate row type to validate mainly for column strategies
+   * @param realTargetRowType       target table row type exclusive virtual columns
+   * @param source                  source node
+   * @param logicalSourceRowType    source node row type
+   * @param logicalTargetRowType    logical target row type, contains only target columns if
+   *                                they are specified or if the sql dialect allows subset insert,
+   *                                make a subset of fields(start from the left first field) whose
+   *                                length is equals with the source row type fields number.
+   */
   private void checkFieldCount(SqlNode node, SqlValidatorTable table,
-      SqlNode source, RelDataType logicalSourceRowType,
-      RelDataType logicalTargetRowType) {
+      List<ColumnStrategy> strategies, RelDataType targetRowTypeToValidate,
+      RelDataType realTargetRowType, SqlNode source,
+      RelDataType logicalSourceRowType, RelDataType logicalTargetRowType) {
     final int sourceFieldCount = logicalSourceRowType.getFieldCount();
     final int targetFieldCount = logicalTargetRowType.getFieldCount();
-    if (sourceFieldCount != targetFieldCount) {
+    final int targetRealFieldCount = realTargetRowType.getFieldCount();
+    if (sourceFieldCount != targetFieldCount
+        && sourceFieldCount != targetRealFieldCount) {
+      // we allow the source row fields equals with either the target row fields num,
+      // or the number of real(exclusive columns that can not insert into) target row fields.
       throw newValidationError(node,
           RESOURCE.unmatchInsertColumn(targetFieldCount, sourceFieldCount));
     }
     // Ensure that non-nullable fields are targeted.
-    final InitializerContext rexBuilder =
-        new InitializerContext() {
-          public RexBuilder getRexBuilder() {
-            return new RexBuilder(typeFactory);
-          }
-
-          public RexNode convertExpression(SqlNode e) {
-            throw new UnsupportedOperationException();
-          }
-        };
-    final List<ColumnStrategy> strategies =
-        table.unwrap(RelOptTable.class).getColumnStrategies();
     for (final RelDataTypeField field : table.getRowType().getFieldList()) {
       final RelDataTypeField targetField =
-          logicalTargetRowType.getField(field.getName(), true, false);
+          targetRowTypeToValidate.getField(field.getName(), true, false);
       switch (strategies.get(field.getIndex())) {
       case NOT_NULLABLE:
         assert !field.getType().isNullable();
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 33bbbdd..15b2502 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -9419,6 +9419,34 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     s.sql(sql1).fails(error);
   }
 
+  @Test public void testInsertTargetTableWithVirtualColumns() {
+    final Sql s = sql("?").withExtendedCatalog();
+    s.sql("insert into VIRTUALCOLUMNS.VC_T1 select a, b, c from VIRTUALCOLUMNS.VC_T2").ok();
+
+    final String sql0 = "insert into ^VIRTUALCOLUMNS.VC_T1^ values(1, 2, 'abc', 3, 4)";
+    final String error0 = "Cannot INSERT into generated column 'D'";
+    s.sql(sql0).fails(error0);
+
+    final String sql1 = "insert into ^VIRTUALCOLUMNS.VC_T1^ values(1, 2, 'abc', DEFAULT, DEFAULT)";
+    s.sql(sql1).ok();
+
+    final String sql2 = "insert into ^VIRTUALCOLUMNS.VC_T1^ values(1, 2, 'abc', DEFAULT)";
+    final String error2 = "(?s).*Number of INSERT target columns \\(5\\) "
+        + "does not equal number of source items \\(4\\).*";
+    s.sql(sql2).fails(error2);
+
+    final String sql3 = "insert into ^VIRTUALCOLUMNS.VC_T1^ "
+        + "values(1, 2, 'abc', DEFAULT, DEFAULT, DEFAULT)";
+    final String error3 = "(?s).*Number of INSERT target columns \\(5\\) "
+        + "does not equal number of source items \\(6\\).*";
+    s.sql(sql3).fails(error3);
+
+    final String sql4 = "insert into VIRTUALCOLUMNS.VC_T1 ^values(1, '2', 'abc')^";
+    final String error4 = "(?s).*Cannot assign to target field 'B' of type BIGINT "
+        + "from source field 'EXPR\\$1' of type CHAR\\(1\\).*";
+    s.sql(sql4).fails(error4);
+  }
+
   @Test public void testInsertFailNullability() {
     tester.checkQueryFails(
         "insert into ^empnullables^ (ename) values ('Kevin')",
diff --git a/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java b/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
index 3bfca7f..77451bc 100644
--- a/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
@@ -32,6 +32,8 @@ import java.util.Arrays;
 final class Fixture {
   final RelDataType intType;
   final RelDataType intTypeNull;
+  final RelDataType bigintType;
+  final RelDataType bigintTypeNull;
   final RelDataType varchar10Type;
   final RelDataType varchar10TypeNull;
   final RelDataType varchar20Type;
@@ -53,6 +55,8 @@ final class Fixture {
   Fixture(RelDataTypeFactory typeFactory) {
     intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
     intTypeNull = typeFactory.createTypeWithNullability(intType, true);
+    bigintType = typeFactory.createSqlType(SqlTypeName.BIGINT);
+    bigintTypeNull = typeFactory.createTypeWithNullability(bigintType, true);
     varchar10Type = typeFactory.createSqlType(SqlTypeName.VARCHAR, 10);
     varchar10TypeNull = typeFactory.createTypeWithNullability(varchar10Type, true);
     varchar20Type = typeFactory.createSqlType(SqlTypeName.VARCHAR, 20);
diff --git a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
index 9fdb321..4311369 100644
--- a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
@@ -113,6 +113,30 @@ public class MockCatalogReaderExtended extends MockCatalogReaderSimple {
     }
     registerTable(structExtendedTypeTable);
 
+    // Defines a table with
+    // schema(A int, B bigint, C varchar(10), D as a + 1 stored, E as b * 3 virtual).
+    MockSchema virtualColumnsSchema = new MockSchema("VIRTUALCOLUMNS");
+    registerSchema(virtualColumnsSchema);
+    final MockTable virtualColumnsTable1 =
+        MockTable.create(this, virtualColumnsSchema, "VC_T1", false, 100,
+            null, new VirtualColumnsExpressionFactory(), false);
+    virtualColumnsTable1.addColumn("A", f.intTypeNull);
+    virtualColumnsTable1.addColumn("B", f.bigintType);
+    virtualColumnsTable1.addColumn("C", f.varchar10Type);
+    virtualColumnsTable1.addColumn("D", f.intTypeNull);
+    virtualColumnsTable1.addColumn("E", f.bigintType);
+    // Same schema with VC_T1 but with different table name.
+    final MockTable virtualColumnsTable2 =
+        MockTable.create(this, virtualColumnsSchema, "VC_T2", false, 100,
+            null, new VirtualColumnsExpressionFactory(), false);
+    virtualColumnsTable2.addColumn("A", f.intTypeNull);
+    virtualColumnsTable2.addColumn("B", f.bigintType);
+    virtualColumnsTable2.addColumn("C", f.varchar10Type);
+    virtualColumnsTable2.addColumn("D", f.intTypeNull);
+    virtualColumnsTable2.addColumn("E", f.bigintType);
+    registerTable(virtualColumnsTable1);
+    registerTable(virtualColumnsTable2);
+
     return this;
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/schema/ColumnStrategy.java b/core/src/test/java/org/apache/calcite/test/catalog/VirtualColumnsExpressionFactory.java
similarity index 50%
copy from core/src/main/java/org/apache/calcite/schema/ColumnStrategy.java
copy to core/src/test/java/org/apache/calcite/test/catalog/VirtualColumnsExpressionFactory.java
index f51175d..13031c2 100644
--- a/core/src/main/java/org/apache/calcite/schema/ColumnStrategy.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/VirtualColumnsExpressionFactory.java
@@ -14,29 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.schema;
+
+package org.apache.calcite.test.catalog;
 
 import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.schema.ColumnStrategy;
+import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
 
-/** Describes how a column gets populated.
- *
- * @see org.apache.calcite.sql2rel.InitializerExpressionFactory#generationStrategy
- * @see RelOptTable#getColumnStrategies()
- */
-public enum ColumnStrategy {
-  /** Column does not have a default value, but does allow null values.
-   * If you don't specify it in an INSERT, it will get a NULL value. */
-  NULLABLE,
-  /** Column does not have a default value, and does not allow nulls.
-   * You must specify it in an INSERT. */
-  NOT_NULLABLE,
-  /** Column has a default value.
-   * If you don't specify it in an INSERT, it will get a NULL value. */
-  DEFAULT,
-  /** Column is computed and stored. You cannot insert into it. */
-  STORED,
-  /** Column is computed and not stored. You cannot insert into it. */
-  VIRTUAL
+/** Define column strategies for the "VIRTUALCOLUMNS" table. */
+public class VirtualColumnsExpressionFactory extends NullInitializerExpressionFactory {
+  @Override public ColumnStrategy generationStrategy(RelOptTable table, int iColumn) {
+    switch (iColumn) {
+    case 3:
+      return ColumnStrategy.STORED;
+    case 4:
+      return ColumnStrategy.VIRTUAL;
+    default:
+      return super.generationStrategy(table, iColumn);
+    }
+  }
 }
 
-// End ColumnStrategy.java
+// End VirtualColumnsExpressionFactory.java
diff --git a/server/src/test/java/org/apache/calcite/test/ServerTest.java b/server/src/test/java/org/apache/calcite/test/ServerTest.java
index 3e4a71b..53e0843 100644
--- a/server/src/test/java/org/apache/calcite/test/ServerTest.java
+++ b/server/src/test/java/org/apache/calcite/test/ServerTest.java
@@ -207,7 +207,6 @@ public class ServerTest {
       // No target column list; too few values provided
       try {
         x = s.executeUpdate("insert into t values (2, 3)");
-        fail("expected error, got " + x);
       } catch (SQLException e) {
         assertThat(e.getMessage(),
             containsString("Number of INSERT target columns (3) does not equal "
@@ -278,8 +277,8 @@ public class ServerTest {
       assertThat(x, is(2));
       try (ResultSet r = s.executeQuery("select sum(i), count(*) from t")) {
         assertThat(r.next(), is(true));
-        assertThat(r.getInt(1), is(19));
-        assertThat(r.getInt(2), is(9));
+        assertThat(r.getInt(1), is(22));
+        assertThat(r.getInt(2), is(10));
         assertThat(r.next(), is(false));
       }
     }