You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@calcite.apache.org by "jamie12221 (Jira)" <ji...@apache.org> on 2020/06/27 08:15:00 UTC

[jira] [Updated] (CALCITE-4046) Check the number of input nodes(org.apache.calcite.interpreter.SetOpNode)

     [ https://issues.apache.org/jira/browse/CALCITE-4046?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

jamie12221 updated CALCITE-4046:
--------------------------------
    Description: 
 

 

 
{code:java}
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;
  }
{code}
 

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

 

The Test Case:

 
{code:java}
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;
    }
  }
}

{code}

  was:
 

 

 
{code:java}
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;
  }
{code}
 

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


> Check the number of input nodes(org.apache.calcite.interpreter.SetOpNode)
> -------------------------------------------------------------------------
>
>                 Key: CALCITE-4046
>                 URL: https://issues.apache.org/jira/browse/CALCITE-4046
>             Project: Calcite
>          Issue Type: Improvement
>          Components: core
>    Affects Versions: 1.23.0
>            Reporter: jamie12221
>            Priority: Trivial
>
>  
>  
>  
> {code:java}
> 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;
>   }
> {code}
>  
> There may be more than two input nodes. If the input is not checked, the result set is wrong.
>  
> The Test Case:
>  
> {code:java}
> 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;
>     }
>   }
> }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)