Uploaded image for project: 'Calcite'
  1. Calcite
  2. CALCITE-4046

In interpreter, set operations with more than 2 inputs give wrong results

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Open
    • Trivial
    • Resolution: Unresolved
    • 1.23.0
    • None
    • core
    • None

    Description

      Check the number of input nodes (org.apache.calcite.interpreter.SetOpNode). 
       

      public class SetOpNode implements Node {
        private final Source leftSource;
        private final Source rightSource;
        private final Sink sink;
        private final SetOp setOp;
      
        public SetOpNode(Compiler compiler, SetOp setOp) {
          leftSource = compiler.source(setOp, 0);
          rightSource = compiler.source(setOp, 1);
          sink = compiler.sink(setOp);
          this.setOp = setOp;
        }
      

       

      There may be more than two input nodes. If the input is not checked, the result set is wrong.

       

      The Test Case:

       

      import org.apache.calcite.DataContext;
      import org.apache.calcite.adapter.java.JavaTypeFactory;
      import org.apache.calcite.adapter.java.ReflectiveSchema;
      import org.apache.calcite.config.CalciteConnectionConfig;
      import org.apache.calcite.config.CalciteConnectionConfigImpl;
      import org.apache.calcite.config.CalciteConnectionProperty;
      import org.apache.calcite.interpreter.Interpreter;
      import org.apache.calcite.jdbc.CalciteSchema;
      import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
      import org.apache.calcite.linq4j.QueryProvider;
      import org.apache.calcite.plan.ConventionTraitDef;
      import org.apache.calcite.plan.RelOptCluster;
      import org.apache.calcite.plan.RelOptPlanner;
      import org.apache.calcite.plan.RelOptTable;
      import org.apache.calcite.plan.volcano.VolcanoPlanner;
      import org.apache.calcite.prepare.CalciteCatalogReader;
      import org.apache.calcite.rel.RelRoot;
      import org.apache.calcite.rel.type.RelDataType;
      import org.apache.calcite.rel.type.RelDataTypeFactory;
      import org.apache.calcite.rex.RexBuilder;
      import org.apache.calcite.schema.SchemaPlus;
      import org.apache.calcite.sql2rel.SqlToRelConverter;
      import org.apache.calcite.tools.RelBuilder;
      import org.junit.Assert;
      
      import java.util.Collections;
      import java.util.List;
      import java.util.Properties;
      
      public class Test {
      
        /***/
        public static class Author {
          public final int id;
          public final String fname;
          public final String lname;
      
          public Author(final int id, final String firstname, final String lastname) {
            this.id = id;
            this.fname = firstname;
            this.lname = lastname;
          }
        }
      
        /***/
        public static class Book {
          public final int id;
          public final String title;
          public final int year;
          public final Author author;
      
          public Book(final int id, final String title, final int year, final Author author) {
            this.id = id;
            this.title = title;
            this.year = year;
            this.author = author;
          }
        }
      
        /***/
        public static class BookStore {
          public final Author[] author = new Author[]{
              new Author(1, "Victor", "Hugo"),
              new Author(2, "Alexandre", "Dumas")
          };
          public final Book[] book = new Book[]{
              new Book(1, "Les Miserables", 1862, author[0]),
              new Book(2, "The Hunchback of Notre-Dame", 1829, author[0]),
              new Book(3, "The Last Day of a Condemned Man", 1829, author[0]),
              new Book(4, "The three Musketeers", 1844, author[1]),
              new Book(5, "The Count of Monte Cristo", 1884, author[1])
          };
        }
      
        @org.junit.Test
        public void example() throws Exception {
          CalciteSchema schema = CalciteSchema.createRootSchema(true);
          schema.add("bs", new ReflectiveSchema(new BookStore()));
          RelDataTypeFactory typeFactory = new JavaTypeFactoryImpl();
          Properties props = new Properties();
          props.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "false");
          CalciteConnectionConfig config = new CalciteConnectionConfigImpl(props);
          CalciteCatalogReader catalogReader = new CalciteCatalogReader(schema,
              Collections.singletonList("bs"),
              typeFactory, config);
          RelOptCluster cluster = newCluster(typeFactory);
          RelBuilder relBuilder = SqlToRelConverter.Config.DEFAULT.getRelBuilderFactory().create(cluster, catalogReader);
          SchemaOnlyDataContext schemaOnlyDataContext = new SchemaOnlyDataContext(schema);
          Assert.assertEquals(5, new Interpreter(schemaOnlyDataContext, relBuilder
                  .scan("Book").build()).count());
          Assert.assertEquals(15, new Interpreter(schemaOnlyDataContext, relBuilder
                  .scan("Book")
                  .scan("Book")
                  .scan("Book")
                  .union(true,3)
                  .build()).count());
        }
      
        private static RelOptCluster newCluster(RelDataTypeFactory factory) {
          RelOptPlanner planner = new VolcanoPlanner();
          planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
          return RelOptCluster.create(planner, new RexBuilder(factory));
        }
      
        private static final RelOptTable.ViewExpander NOOP_EXPANDER = new RelOptTable.ViewExpander() {
          @Override public RelRoot expandView(final RelDataType rowType, final String queryString,
              final List<String> schemaPath,
              final List<String> viewPath) {
            return null;
          }
        };
      
        /**
         * A simple data context only with schema information.
         */
        private static final class SchemaOnlyDataContext implements DataContext {
          private final SchemaPlus schema;
      
          SchemaOnlyDataContext(CalciteSchema calciteSchema) {
            this.schema = calciteSchema.plus();
          }
      
          @Override public SchemaPlus getRootSchema() {
            return schema;
          }
      
          @Override public JavaTypeFactory getTypeFactory() {
            return  new JavaTypeFactoryImpl();
          }
      
          @Override public QueryProvider getQueryProvider() {
            return null;
          }
      
          @Override public Object get(final String name) {
            return null;
          }
        }
      }
      
      

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              jamie12221 jamie12221
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated: