Query hints support and example of extended SQL syntax

2019-10-28 Thread Vladimir Ozerov
Hi colleagues,

I am working on Calcite integration with Hazelcast. One particular problem
is that we need query hints. I found a couple of tickets [1] [2], and it
seems that hints are not readily available at the moment. So I have two
questions:
1) Is it true that there is no built-in way to provide query hints at the
moment? Are there any plans to finally add hints to the engine?
2) As a possible solution, I am considering extending SELECT syntax with a
custom parser extension. Do you know any example of extended SELECT syntax
in existing Calcite integrations? I found an example of SqlInsert command
extension in Apache Flink, but maybe somebody already extended SELECT as
well?

Thank you,
Vladimir.

[1] https://issues.apache.org/jira/browse/CALCITE-482
[2] https://issues.apache.org/jira/browse/CALCITE-495


Problem with converters and possibly rule matching

2019-10-28 Thread Vladimir Ozerov
Hi colleagues,

We are building a Calcite-based optimizer for Hazelcast, and I have some
problems understanding Calcite's logic with respect to converters. Let me
briefly explain the problem.

We have an execution backend, so we do not need Bindable or Enumerable.
Instead, we would like to use Calcite to convert original SQL to a tree
with our own convention, then convert it to our internal representation,
and finally, execute.

We started with looking at other Calcite integrations and eventually came
to a classical two-phase optimization approach. We have two internal
conventions - LOGICAL and PHYSICAL. The goal is to optimize the tree as
follows:
1) NONE -> LOGICAL - heuristical optimizations
2) LOGICAL -> PHYSICAL - cost-based planning

Suppose that after the first phase I have the following tree of our own
operators:
HZLogicalRoot
-> HZLogicalProject
  -> HZLogicalScan

For this specific case, there is not much to optimize, so we only need to
transition to physical nodes and do some boilerplate with traits
propagation:
HZPhysicalRoot
-> HZPhysicalProject
  -> HZPhysicalScan

In order to achieve this, I define three rules, which just do a conversion
of relevant nodes. Volcano optimizer is used.

Now, the problem - somehow it works only when I override
Convention.Impl.canConvertConvention to true for our PHYSICAL convention,
but that blows the search space and the same rules are called many times. A
lot of time is spent on endless PHYSICAL -> LOGICAL conversions, which are
of no use.

If I change canConvertConvention to false, then rules are called a sensible
number of times, but cannot produce a complete PHYSICAL tree. Here is how
it works:
1) "Root" rule is invoked, which converts "HZLogicalRoot" to
"HZPhysicalRoot"
2) "Project" rule is invoked, but do not produce any transformations, since
it needs Scan distribution, which is not known yet. This desired behavior
at this point.
3) "Scan" rule is invoked, "HZLogicalScan" is converted to
"HZPhysicalScan". Distribution is resolved
4) At this point, we have [LogicalRoot, PhysicalRoot] -> [LogicalProject]
-> [LogicalScan, PhysicalScan] sets . I expect that since new scan was
installed, the "Project" rule will be fired again. This time we know the
distribution, so the transformation is possible. But the rule is not called
and we fail with an error.

So my questions are:
1) What is the real role of converters in this process? For some reason,
when unnecessary (from a logical standpoint) PHYSICAL -> LOGICAL conversion
is allowed, even complex plans could be built. And Drill does it for some
reason. But it costs multiple additional invocations of the same rules. Are
there any docs or presentations explaining the mechanics behind?
2) What are the minimum requirements, that will allow a rule on the parent
to be fired again after it's child node has changed?

I can provide any additional information, source code or even working
example of this problem if needed. I don't want to bother you with it at
the moment, because it feels like I miss something very simple.

Would appreciate your help.

Regards,
Vladimir.


Re: Query hints support and example of extended SQL syntax

2019-10-28 Thread Vladimir Ozerov
Hi Danny,

I looked at your design proposal and it looks great. All common hint cases
that we may possibly need are covered there. Thank you.

I hope the community will release it soon since hints are essential for
production-grade systems. From what I see in other projects, such as Flink
or Drill, they have to rely on properties set somewhere else
(configuration, session), to achieve the same goals, which is not
user-friendly.

Regards,
Vladimir.

пн, 28 окт. 2019 г. в 13:26, Danny Chan :

> Here is the discussion mailing list [1]
>
> [1]
> https://ponymail-vm.apache.org/_GUI_/thread.html/db3799d70232ec85e294bb8885431a3f5c88bb28f4fd3337368b1480@%3Cdev.calcite.apache.org%3E
>
> Best,
> Danny Chan
> 在 2019年10月28日 +0800 PM5:09,Vladimir Ozerov ,写道:
> > Hi colleagues,
> >
> > I am working on Calcite integration with Hazelcast. One particular
> problem
> > is that we need query hints. I found a couple of tickets [1] [2], and it
> > seems that hints are not readily available at the moment. So I have two
> > questions:
> > 1) Is it true that there is no built-in way to provide query hints at the
> > moment? Are there any plans to finally add hints to the engine?
> > 2) As a possible solution, I am considering extending SELECT syntax with
> a
> > custom parser extension. Do you know any example of extended SELECT
> syntax
> > in existing Calcite integrations? I found an example of SqlInsert command
> > extension in Apache Flink, but maybe somebody already extended SELECT as
> > well?
> >
> > Thank you,
> > Vladimir.
> >
> > [1] https://issues.apache.org/jira/browse/CALCITE-482
> > [2] https://issues.apache.org/jira/browse/CALCITE-495
>


Re: Problem with converters and possibly rule matching

2019-10-29 Thread Vladimir Ozerov
Hi everybody,

First of all, thank you for answers and suggestions. Let me address them
briefly:
1) I use two conventions at the moment, LOGICAL and PHYSICAL. I agree with
you that this might be overkill, and there is a chance that in the final
solution we may end up with only one. But meanwhile having this separation
seems handy, because on the first stage I enforce the optimizer to
propagate NONE -> LOGICAL conversions. Then I have a clean logical tree
*before* any physical distribution stuff is involved, which I can use for
internal post-processing before going logical. I would propose to keep it
out of scope at the moment. Let's just consider that the goal is to convert
from one convention to another.
2) Operands with "any" matchers are already used
3) The reason why I would like to avoid the "Project" rule fire on the
first run, is that it doesn't enforce any distribution on its child
("Scan"). Instead, it needs to derive the distribution from the scan.

To make the problem more clear, let me prepare a simple reproducer for the
issue.

Regards,
Vladimir.

вт, 29 окт. 2019 г. в 10:01, Seliverstov Igor :

> Vladimir,
>
> I guess Project rule doesn't have a child matcher. Put into it child "any"
> match rule and it will apply on each child node transformation.
>
> Regards,
> Igor
>
>
> вт, 29 окт. 2019 г., 7:07 Danny Chan :
>
> > Vladimir, all you need to do is to change the convention of the root
> node,
> > the volcano would propagate the convention to all its input nodes when
> > registering them to the planner. You can take this code [1] for
> reference :)
> >
> > [1]
> >
> https://github.com/apache/calcite/blob/1ef2821695ca6e10fbad7b8efe7246c4a20143af/core/src/main/java/org/apache/calcite/tools/Programs.java#L324
> >
> > Best,
> > Danny Chan
> > 在 2019年10月29日 +0800 AM5:24,dev@calcite.apache.org,写道:
> > >
> > > n of the scan.
> >
>


Re: Problem with converters and possibly rule matching

2019-10-29 Thread Vladimir Ozerov
HYSICAL.SINGLETON.[](input=ProjectPhysicalRel#29,hashFields=[0])
// Not needed!
ProjectPhysicalRel.PHYSICAL.ANY.[](input=MapScanPhysicalRel#25,...))
  MapScanPhysicalRel.PHYSICAL.REPLICATED.[](table=[t],projects=[0])

So as you can see, it is possible to force Calcite to do what I need in
this specific case - to re-execute the rules of parent nodes after the
child has changed. But this workarounds creates some unnecessary
intermediate nodes. With complex queries, this may increase planning time
significantly.

My main question is - what key part of converters logic or Calcite
framework I am missing? Why I cannot force the Calcite to call parent rules
out of the box, but converters seem to do that at the cost of additional
litter?

Thank you very much in advance.
Vladimir.

[1]
https://github.com/devozerov/calcite-optimizer/blob/master/src/test/java/devozerov/OptimizerTest.java
[2]
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/distribution/DistributionTrait.java
[3]
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/distribution/DistributionTraitDef.java
[4]
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/RootPhysicalRule.java

[5]
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/MapScanPhysicalRule.java
[6]
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/ProjectPhysicalRule.java
[7]
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/HazelcastConventions.java#L38

вт, 29 окт. 2019 г. в 11:26, Vladimir Ozerov :

> Hi everybody,
>
> First of all, thank you for answers and suggestions. Let me address them
> briefly:
> 1) I use two conventions at the moment, LOGICAL and PHYSICAL. I agree with
> you that this might be overkill, and there is a chance that in the final
> solution we may end up with only one. But meanwhile having this separation
> seems handy, because on the first stage I enforce the optimizer to
> propagate NONE -> LOGICAL conversions. Then I have a clean logical tree
> *before* any physical distribution stuff is involved, which I can use for
> internal post-processing before going logical. I would propose to keep it
> out of scope at the moment. Let's just consider that the goal is to convert
> from one convention to another.
> 2) Operands with "any" matchers are already used
> 3) The reason why I would like to avoid the "Project" rule fire on the
> first run, is that it doesn't enforce any distribution on its child
> ("Scan"). Instead, it needs to derive the distribution from the scan.
>
> To make the problem more clear, let me prepare a simple reproducer for the
> issue.
>
> Regards,
> Vladimir.
>
> вт, 29 окт. 2019 г. в 10:01, Seliverstov Igor :
>
>> Vladimir,
>>
>> I guess Project rule doesn't have a child matcher. Put into it child "any"
>> match rule and it will apply on each child node transformation.
>>
>> Regards,
>> Igor
>>
>>
>> вт, 29 окт. 2019 г., 7:07 Danny Chan :
>>
>> > Vladimir, all you need to do is to change the convention of the root
>> node,
>> > the volcano would propagate the convention to all its input nodes when
>> > registering them to the planner. You can take this code [1] for
>> reference :)
>> >
>> > [1]
>> >
>> https://github.com/apache/calcite/blob/1ef2821695ca6e10fbad7b8efe7246c4a20143af/core/src/main/java/org/apache/calcite/tools/Programs.java#L324
>> >
>> > Best,
>> > Danny Chan
>> > 在 2019年10月29日 +0800 AM5:24,dev@calcite.apache.org,写道:
>> > >
>> > > n of the scan.
>> >
>>
>


Re: Problem with converters and possibly rule matching

2019-10-29 Thread Vladimir Ozerov
Hi Vladimir,

I am sorry. Pushed, it works now.

вт, 29 окт. 2019 г. в 14:41, Vladimir Sitnikov :

> > mvn clean test
>
> [ERROR] The goal you specified requires a project to execute but there is
> no POM in this directory
>
> Vladimir, please push missing files
>
> Vladimir
>


Re: Problem with converters and possibly rule matching

2019-10-30 Thread Vladimir Ozerov
Hi Igor,

Yes, I came to the same conclusion, thank you. This is how it basically
happens when converters are disabled:
1) We start with initial tree: [LogicalProject] <- [LogicalScan]
2) Then we convert LogicalScan to PhysicalScan, so it is added to the
set: [LogicalProject]
<- [LogicalScan, PhysicalScan]
3) Finally, when it is time to fire a rule for PhysicalScan, we try to get
parents of that scan set with traits of the PhysicalScan. Since there are
no such parents (we skipped it intentionally), the rule is not queued.

But when converters are enabled, a converter rel is created: [LogicalProject]
<- [LogicalScan, PhysicalScan, ConverterFromPhysicalToLogical]. No rules
are fired for PhysicalScan again, but they are fired for converter since it
has the necessary LOGICAL trait.

It makes sense, that converters essentially allow forcing rule invocation
on parents, even if the child was created with different traits. But it
seems that this mechanism may be too heavy for complex queries because it
literally creates hundreds of new converter rels and triggers rules over
and over again.

We need some fine-grained alternative. Basically, what would be enough for
me is to let the planner know somehow: "I created that rel, and I want you
to execute parent rules not only using its trait but also on this and those
traits."
Is there any API in Calcite which allows doing this without creating a new
rel node?

Regards,
Vladimir.


ср, 30 окт. 2019 г. в 09:25, Seliverstov Igor :

> Vladimir,
>
> Probably it'll help you:
>
> Seems the cause of issue in RelSubset.getParentRels()  The check used when
> the planner schedules newly matched rules after successful transformation
> (see VolcanoRuleCall.matchRecurse), it prevents the parent rule be applied
> once again (here your logical project with an input having ANY distribution
> doesn't satisfy a transformed input traits).
>
> In our case we use another workaround, so there are also much more
> transformations than we wanted, so the desired rule is triggered.
>
>
> вт, 29 окт. 2019 г., 14:46 Vladimir Ozerov :
>
> > Hi Vladimir,
> >
> > I am sorry. Pushed, it works now.
> >
> > вт, 29 окт. 2019 г. в 14:41, Vladimir Sitnikov <
> > sitnikov.vladi...@gmail.com
> > >:
> >
> > > > mvn clean test
> > >
> > > [ERROR] The goal you specified requires a project to execute but there
> is
> > > no POM in this directory
> > >
> > > Vladimir, please push missing files
> > >
> > > Vladimir
> > >
> >
>


Re: Problem with converters and possibly rule matching

2019-10-30 Thread Vladimir Ozerov
One funny hack which helped me is manual registration of a fake RelNode
with desired traits through VolcanoPlanner.register() method. But again,
this leads to trashing. What could really help is a call to
VolcanoPlanner.fireRules() with desired rel. But this doesn't work out of
the box since some internals of the rule queue needs to be adjusted.

What does the community think about adding a method which will re-add rules
applicable to the specific RelNode to the rule queue?

ср, 30 окт. 2019 г. в 17:00, Vladimir Ozerov :

> Hi Igor,
>
> Yes, I came to the same conclusion, thank you. This is how it basically
> happens when converters are disabled:
> 1) We start with initial tree: [LogicalProject] <- [LogicalScan]
> 2) Then we convert LogicalScan to PhysicalScan, so it is added to the
> set: [LogicalProject] <- [LogicalScan, PhysicalScan]
> 3) Finally, when it is time to fire a rule for PhysicalScan, we try to get
> parents of that scan set with traits of the PhysicalScan. Since there are
> no such parents (we skipped it intentionally), the rule is not queued.
>
> But when converters are enabled, a converter rel is created: [LogicalProject]
> <- [LogicalScan, PhysicalScan, ConverterFromPhysicalToLogical]. No rules
> are fired for PhysicalScan again, but they are fired for converter since
> it has the necessary LOGICAL trait.
>
> It makes sense, that converters essentially allow forcing rule invocation
> on parents, even if the child was created with different traits. But it
> seems that this mechanism may be too heavy for complex queries because it
> literally creates hundreds of new converter rels and triggers rules over
> and over again.
>
> We need some fine-grained alternative. Basically, what would be enough for
> me is to let the planner know somehow: "I created that rel, and I want you
> to execute parent rules not only using its trait but also on this and those
> traits."
> Is there any API in Calcite which allows doing this without creating a new
> rel node?
>
> Regards,
> Vladimir.
>
>
> ср, 30 окт. 2019 г. в 09:25, Seliverstov Igor :
>
>> Vladimir,
>>
>> Probably it'll help you:
>>
>> Seems the cause of issue in RelSubset.getParentRels()  The check used when
>> the planner schedules newly matched rules after successful transformation
>> (see VolcanoRuleCall.matchRecurse), it prevents the parent rule be applied
>> once again (here your logical project with an input having ANY
>> distribution
>> doesn't satisfy a transformed input traits).
>>
>> In our case we use another workaround, so there are also much more
>> transformations than we wanted, so the desired rule is triggered.
>>
>>
>> вт, 29 окт. 2019 г., 14:46 Vladimir Ozerov :
>>
>> > Hi Vladimir,
>> >
>> > I am sorry. Pushed, it works now.
>> >
>> > вт, 29 окт. 2019 г. в 14:41, Vladimir Sitnikov <
>> > sitnikov.vladi...@gmail.com
>> > >:
>> >
>> > > > mvn clean test
>> > >
>> > > [ERROR] The goal you specified requires a project to execute but
>> there is
>> > > no POM in this directory
>> > >
>> > > Vladimir, please push missing files
>> > >
>> > > Vladimir
>> > >
>> >
>>
>


Re: Problem with converters and possibly rule matching

2019-10-30 Thread Vladimir Ozerov
“e pension” == “expand conversion” :)

ср, 30 окт. 2019 г. в 18:46, Vladimir Ozerov :

> Yes, that may work. Even if e pension rule is used, for the most cases it
> will not trigger any real conversions, since we are moving from abstract
> convention to physical, and created converters will have the opposite trait
> direction (from physical to abstract).
>
> But again - ideally we only need to re-trigger the rules for a specific
> node, no more than that. So API support like
> “VolcanoPlanner.forceRules(RelNode)” would be very convenient.
>
> What do you think?
>
> ср, 30 окт. 2019 г. в 17:56, Seliverstov Igor :
>
>> I considered manual rules calling too, for now we use abstract converters
>> +
>> ExpandConversionRule for exchanges producing.
>>
>> You may create such converters manually (checking appropriate subset) this
>> case you may reduce created converters count, also, a converter is a quite
>> special node, that does almost nothing (without corresponding rule) it may
>> be used just as a rule trigger.
>>
>> Regards,
>> Igor
>>
>> ср, 30 окт. 2019 г., 17:31 Vladimir Ozerov :
>>
>> > One funny hack which helped me is manual registration of a fake RelNode
>> > with desired traits through VolcanoPlanner.register() method. But again,
>> > this leads to trashing. What could really help is a call to
>> > VolcanoPlanner.fireRules() with desired rel. But this doesn't work out
>> of
>> > the box since some internals of the rule queue needs to be adjusted.
>> >
>> > What does the community think about adding a method which will re-add
>> rules
>> > applicable to the specific RelNode to the rule queue?
>> >
>> > ср, 30 окт. 2019 г. в 17:00, Vladimir Ozerov :
>> >
>> > > Hi Igor,
>> > >
>> > > Yes, I came to the same conclusion, thank you. This is how it
>> basically
>> > > happens when converters are disabled:
>> > > 1) We start with initial tree: [LogicalProject] <- [LogicalScan]
>> > > 2) Then we convert LogicalScan to PhysicalScan, so it is added to the
>> > > set: [LogicalProject] <- [LogicalScan, PhysicalScan]
>> > > 3) Finally, when it is time to fire a rule for PhysicalScan, we try to
>> > get
>> > > parents of that scan set with traits of the PhysicalScan. Since there
>> are
>> > > no such parents (we skipped it intentionally), the rule is not queued.
>> > >
>> > > But when converters are enabled, a converter rel is created:
>> > [LogicalProject]
>> > > <- [LogicalScan, PhysicalScan, ConverterFromPhysicalToLogical]. No
>> rules
>> > > are fired for PhysicalScan again, but they are fired for converter
>> since
>> > > it has the necessary LOGICAL trait.
>> > >
>> > > It makes sense, that converters essentially allow forcing rule
>> invocation
>> > > on parents, even if the child was created with different traits. But
>> it
>> > > seems that this mechanism may be too heavy for complex queries
>> because it
>> > > literally creates hundreds of new converter rels and triggers rules
>> over
>> > > and over again.
>> > >
>> > > We need some fine-grained alternative. Basically, what would be enough
>> > for
>> > > me is to let the planner know somehow: "I created that rel, and I want
>> > you
>> > > to execute parent rules not only using its trait but also on this and
>> > those
>> > > traits."
>> > > Is there any API in Calcite which allows doing this without creating a
>> > new
>> > > rel node?
>> > >
>> > > Regards,
>> > > Vladimir.
>> > >
>> > >
>> > > ср, 30 окт. 2019 г. в 09:25, Seliverstov Igor :
>> > >
>> > >> Vladimir,
>> > >>
>> > >> Probably it'll help you:
>> > >>
>> > >> Seems the cause of issue in RelSubset.getParentRels()  The check used
>> > when
>> > >> the planner schedules newly matched rules after successful
>> > transformation
>> > >> (see VolcanoRuleCall.matchRecurse), it prevents the parent rule be
>> > applied
>> > >> once again (here your logical project with an input having ANY
>> > >> distribution
>> > >> doesn't satisfy a transformed input traits).
>> > >>
>> > >> In our case we use another workaround, so there are also much more
>> > >> transformations than we wanted, so the desired rule is triggered.
>> > >>
>> > >>
>> > >> вт, 29 окт. 2019 г., 14:46 Vladimir Ozerov :
>> > >>
>> > >> > Hi Vladimir,
>> > >> >
>> > >> > I am sorry. Pushed, it works now.
>> > >> >
>> > >> > вт, 29 окт. 2019 г. в 14:41, Vladimir Sitnikov <
>> > >> > sitnikov.vladi...@gmail.com
>> > >> > >:
>> > >> >
>> > >> > > > mvn clean test
>> > >> > >
>> > >> > > [ERROR] The goal you specified requires a project to execute but
>> > >> there is
>> > >> > > no POM in this directory
>> > >> > >
>> > >> > > Vladimir, please push missing files
>> > >> > >
>> > >> > > Vladimir
>> > >> > >
>> > >> >
>> > >>
>> > >
>> >
>>
>


Re: Problem with converters and possibly rule matching

2019-10-30 Thread Vladimir Ozerov
Yes, that may work. Even if e pension rule is used, for the most cases it
will not trigger any real conversions, since we are moving from abstract
convention to physical, and created converters will have the opposite trait
direction (from physical to abstract).

But again - ideally we only need to re-trigger the rules for a specific
node, no more than that. So API support like
“VolcanoPlanner.forceRules(RelNode)” would be very convenient.

What do you think?

ср, 30 окт. 2019 г. в 17:56, Seliverstov Igor :

> I considered manual rules calling too, for now we use abstract converters +
> ExpandConversionRule for exchanges producing.
>
> You may create such converters manually (checking appropriate subset) this
> case you may reduce created converters count, also, a converter is a quite
> special node, that does almost nothing (without corresponding rule) it may
> be used just as a rule trigger.
>
> Regards,
> Igor
>
> ср, 30 окт. 2019 г., 17:31 Vladimir Ozerov :
>
> > One funny hack which helped me is manual registration of a fake RelNode
> > with desired traits through VolcanoPlanner.register() method. But again,
> > this leads to trashing. What could really help is a call to
> > VolcanoPlanner.fireRules() with desired rel. But this doesn't work out of
> > the box since some internals of the rule queue needs to be adjusted.
> >
> > What does the community think about adding a method which will re-add
> rules
> > applicable to the specific RelNode to the rule queue?
> >
> > ср, 30 окт. 2019 г. в 17:00, Vladimir Ozerov :
> >
> > > Hi Igor,
> > >
> > > Yes, I came to the same conclusion, thank you. This is how it basically
> > > happens when converters are disabled:
> > > 1) We start with initial tree: [LogicalProject] <- [LogicalScan]
> > > 2) Then we convert LogicalScan to PhysicalScan, so it is added to the
> > > set: [LogicalProject] <- [LogicalScan, PhysicalScan]
> > > 3) Finally, when it is time to fire a rule for PhysicalScan, we try to
> > get
> > > parents of that scan set with traits of the PhysicalScan. Since there
> are
> > > no such parents (we skipped it intentionally), the rule is not queued.
> > >
> > > But when converters are enabled, a converter rel is created:
> > [LogicalProject]
> > > <- [LogicalScan, PhysicalScan, ConverterFromPhysicalToLogical]. No
> rules
> > > are fired for PhysicalScan again, but they are fired for converter
> since
> > > it has the necessary LOGICAL trait.
> > >
> > > It makes sense, that converters essentially allow forcing rule
> invocation
> > > on parents, even if the child was created with different traits. But it
> > > seems that this mechanism may be too heavy for complex queries because
> it
> > > literally creates hundreds of new converter rels and triggers rules
> over
> > > and over again.
> > >
> > > We need some fine-grained alternative. Basically, what would be enough
> > for
> > > me is to let the planner know somehow: "I created that rel, and I want
> > you
> > > to execute parent rules not only using its trait but also on this and
> > those
> > > traits."
> > > Is there any API in Calcite which allows doing this without creating a
> > new
> > > rel node?
> > >
> > > Regards,
> > > Vladimir.
> > >
> > >
> > > ср, 30 окт. 2019 г. в 09:25, Seliverstov Igor :
> > >
> > >> Vladimir,
> > >>
> > >> Probably it'll help you:
> > >>
> > >> Seems the cause of issue in RelSubset.getParentRels()  The check used
> > when
> > >> the planner schedules newly matched rules after successful
> > transformation
> > >> (see VolcanoRuleCall.matchRecurse), it prevents the parent rule be
> > applied
> > >> once again (here your logical project with an input having ANY
> > >> distribution
> > >> doesn't satisfy a transformed input traits).
> > >>
> > >> In our case we use another workaround, so there are also much more
> > >> transformations than we wanted, so the desired rule is triggered.
> > >>
> > >>
> > >> вт, 29 окт. 2019 г., 14:46 Vladimir Ozerov :
> > >>
> > >> > Hi Vladimir,
> > >> >
> > >> > I am sorry. Pushed, it works now.
> > >> >
> > >> > вт, 29 окт. 2019 г. в 14:41, Vladimir Sitnikov <
> > >> > sitnikov.vladi...@gmail.com
> > >> > >:
> > >> >
> > >> > > > mvn clean test
> > >> > >
> > >> > > [ERROR] The goal you specified requires a project to execute but
> > >> there is
> > >> > > no POM in this directory
> > >> > >
> > >> > > Vladimir, please push missing files
> > >> > >
> > >> > > Vladimir
> > >> > >
> > >> >
> > >>
> > >
> >
>


Re: Problem with converters and possibly rule matching

2019-10-30 Thread Vladimir Ozerov
Hi Stamatis,

The problem that the presented reproducer is a very simplified version of
what is actually needed. In reality, there are several distribution types -
PARTITIONED, REPLICATED, SINGLETON. To make things worse, PARTITIONED may
or may not have concrete distribution fields. In theory, I can create one
transformation per distribution type, but that would increase plan space
significantly. In my sample "Root <- Project <- Scan" plan, there is no
room for optimization at all, we only need to convert the nodes and
propagate traits. But if I follow the proposed approach, the planner would
create three equivalent paths. For complex queries, this may increase
optimization time significantly.

What I need instead is to gradually convert and optimize nodes *bottom-up*,
instead of top-bottom. That is, create a physical scan first, then create a
physical project on top of it, etc. But at the same time, some rules still
require the top-bottom approach. So essentially I need the optimizer to do
both. Abstract converters help me establish bottom-up preparation but do
this at the cost of considering too many trait pairs, and as a result, also
pollute the search space.

To the contrast, precise command "I transformed the node A to node B,
please re-trigger the rules for A's parents" would allow us to re-trigger
only required rules, without adding more nodes.

Does it make sense?

Regards,
Vladimir.

ср, 30 окт. 2019 г. в 21:02, Stamatis Zampetakis :

> I admit that I didn't thoroughly read the exchanges so far but forcing the
> rules trigger does not seem the right approach.
>
> Other than that I would like to backtrack a bit to point 4.3 raised by
> Vladimir.
>
> "ProjectPhysicalRule [6] - transforms logical project to physical
> project *ONLY* if there is an underlying physical input with REPLICATED or
> SINGLETON distribution"
>
> The rule could be modified to do the following two transformations:
> 1. Create a physical project and require the input to be REPLICATED.
> 2. Create a physical project and require
> the input to be SINGLETON.
>
> I would assume that afterwards when your scan rule fires it should go to
> the appropriate subset and give you back the desired plan. Is there a
> problem with this approach?
>
> Best,
> Stamatis
>
> On Wed, Oct 30, 2019, 5:52 PM Seliverstov Igor 
> wrote:
>
> > Unfortunately it requires package-private API usage.
> >
> > Best solution there if Calcite provide it for end users.
> >
> > ср, 30 окт. 2019 г., 18:48 Vladimir Ozerov :
> >
> > > “e pension” == “expand conversion” :)
> > >
> > > ср, 30 окт. 2019 г. в 18:46, Vladimir Ozerov :
> > >
> > > > Yes, that may work. Even if e pension rule is used, for the most
> cases
> > it
> > > > will not trigger any real conversions, since we are moving from
> > abstract
> > > > convention to physical, and created converters will have the opposite
> > > trait
> > > > direction (from physical to abstract).
> > > >
> > > > But again - ideally we only need to re-trigger the rules for a
> specific
> > > > node, no more than that. So API support like
> > > > “VolcanoPlanner.forceRules(RelNode)” would be very convenient.
> > > >
> > > > What do you think?
> > > >
> > > > ср, 30 окт. 2019 г. в 17:56, Seliverstov Igor  >:
> > > >
> > > >> I considered manual rules calling too, for now we use abstract
> > > converters
> > > >> +
> > > >> ExpandConversionRule for exchanges producing.
> > > >>
> > > >> You may create such converters manually (checking appropriate
> subset)
> > > this
> > > >> case you may reduce created converters count, also, a converter is a
> > > quite
> > > >> special node, that does almost nothing (without corresponding rule)
> it
> > > may
> > > >> be used just as a rule trigger.
> > > >>
> > > >> Regards,
> > > >> Igor
> > > >>
> > > >> ср, 30 окт. 2019 г., 17:31 Vladimir Ozerov :
> > > >>
> > > >> > One funny hack which helped me is manual registration of a fake
> > > RelNode
> > > >> > with desired traits through VolcanoPlanner.register() method. But
> > > again,
> > > >> > this leads to trashing. What could really help is a call to
> > > >> > VolcanoPlanner.fireRules() with desired rel. But this doesn't work
> > out
> > > >> of
> > > >> > the box since some internal

[DISCUSS] Proposal to add API to force rules matching specific rels

2019-10-31 Thread Vladimir Ozerov
Hi colleagues,

I would like to discuss with the community the possibility of adding a new
public method to VolcanoPlanner which will forcefully re-trigger the rules
for the specific rel. This is a follow up of a discussion started in the
other thread [1].

**Problem statement**
When converting between conventions during optimization VolcanoPlanner
prefers the top-bottom approach, so that the nodes are converted from the
root. But in some cases, the intermediate node must be converted after its
children. This is especially true for distributed SQL engines, which rely
on distribution traits during the optimization process: it is not possible
to efficiently choose a proper physical implementation of a parent node
unless the physical representation of a child node is known.

It seems that presently VolcanoPlanner cannot address such cases by
default. Consider that we have two nodes and associated rules which convert
them to physical counterparts:
[LogicalParent <- LogicalChild]
The parent should be converted after the child. When the child is
converted, the physical node is created:
[LogicalParent <- {LogicalChild, PhysicalChild}]
In order to finish the optimization process, we need to convert the parent.
But parent rules are not fired, because PhysicalChild has traits
incompatible with LogicalParent.

Presently the problem could be solved in two ways:
1) Always produce conversions when going top-down. In this case, I first
create a physical parent, then a physical child. The problem is that
created parent is not optimal because it didn't know child distribution at
the time of creation. So the full flow would be: create a bad parent,
create a good child, create a good parent.
1.1) [LogicalParent <- LogicalChild]
1.2) [{LogicalParent, PhysicalParentBad} <- LogicalChild]
1.3) [{LogicalParent, PhysicalParentBad} <- {LogicalChild, PhysicalChild}]
1.4) [{LogicalParent, PhysicalParentBad, PhysicalParentGood} <-
{LogicalChild, PhysicalChild}]
What is worse, the creation of a not optimal parent will trigger rules on
its parent, which in turn may create a not optimal parent-parent node, etc.

2) Make sure that your convention returns true for
Convention.canConvertConvention. In this case, every time a new rel is
added to a RelSet, its traitset will be compared to traitsets of all other
rels in the set. For every pair of traitset we may ask the engine to create
a relevant AbstractConverter. The net effect is that "physical-to-logical"
converter is created, which re-triggers the rule on the logical parent
since their conventions are compatible:
2.1) [LogicalParent  <- LogicalChild]
2.2) [LogicalParent <- {LogicalChild, PhysicalChild}]
2.3) [LogicalParent <- {LogicalChild, PhysicalChild,
PhysicalToLogicalConverter}]
2.4) [{LogicalParent, PhysicalParent} <- {LogicalChild, PhysicalChild,
PhysicalToLogicalConverter}]

The problem with that approach is that it is too coarse-grained since we
operate on traitsets rather than rels. As a result, additional memory and
CPU pressure are introduced because usually too many converters are
created, which triggers the rules over and over again.

**Affected products**
At the moment two distributed engines are being developed for Hazelcast and
Apache Ignite. Both require bottom-up optimization and currently rely on
the second workaround.
Another example is Apache Drill. I do not know whether its community is
concerned with the issue, but it also uses bottom-up optimization for many
rules and employs both the aforementioned workarounds. As a result, I guess
that Drill's optimizer also creates too many rels during optimization and
suffer from huge search space. Please correct me if I am wrong.

**Proposal**
The key problem is that there is no way to re-fire rules on a specific
node. The original problem could have been solved, if it would be possible
to re-fire rules on a LogicalParent without creating any additional rels.
That would lead to a clear optimization flow:
2.1) [LogicalParent  <- LogicalChild]
2.2) [LogicalParent <- {LogicalChild, PhysicalChild}]
2.3) [{LogicalParent, PhysicalParent} <- {LogicalChild, PhysicalChild}]

We can add the following method to VolcanoPlanner (RelOptPlanner?)
interface:
void fireRules(RelNode rel)

This method will fire the rules on a passed node in a deferred mode as if
it was the new node just added to the optimizer. This would require slight
changes to RuleQueue.addMatch method, and possibly some other places.

Usage example:
class PhysicalChildRule extends RelOptRule {
void onMatch(RelOptRuleCall call) {
LogicalChild logicalRel = call.get(0);
PhysicalChild physicalRel = ...;

call.transformTo(physicalRel);
optimizer.fireRules(logicalRel);
}
}

What does the community think about such a method? Does it make any sense
to you? If not, do you aware of any other ways on how to organize bottom-up
optimization with VolcanoPlanner without the creation of additional rels?

If the community is OK in general, I can creat

Re: [DISCUSS] Proposal to add API to force rules matching specific rels

2019-10-31 Thread Vladimir Ozerov
Hi Danny,

Thank you very much for the links. What is described here is pretty much
similar to the problem I describe. Especially the discussion about trait
propagation, as this is basically what I need - to explore potential traits
of children before optimizing parents. And this is basically what Drill
already does with it's SubsetTransformer:
1) There is a SubsetTransformer interface, which iterates over physical
relations of the given subset [1]
2) If you want to make a physical project, you iterate over physical
relations of the input subset and create possible physical projects [2]
3) But if you cannot find any physical input, then you trigger creation of
a "bad" physical project, which is very likely to have poor cost because it
cannot take advantage of input's distribution information [3]
So, step 2 - is a trait set propagation which is needed by many
distributed engines. Step 3 - an attempt to workaround current
VolcanoPlanner behavior, when a parent rule is fired only if produced child
node has compatible trait set.

I do not know Calcite's architecture that good but at first glance, the
proposed ability to re-fire rules of a specific Rel seems good enough. It
doesn't expand search space, because no new nodes are created, and it seems
to be relatively simple to implement.

[1]
https://github.com/apache/drill/blob/1.16.0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
[2]
https://github.com/apache/drill/blob/1.16.0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java#L66
[3]
https://github.com/apache/drill/blob/1.16.0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java#L69

чт, 31 окт. 2019 г. в 12:21, Danny Chan :

> Thanks Vladimir for bringing up this discussion !
>
> Did you notice that there is a JIRA issue about this problem ? [1] Also a
> discussion about how to propagate the traits [2]
>
> [1] https://issues.apache.org/jira/browse/CALCITE-2970
> [2]
> https://ponymail-vm.apache.org/_GUI_/thread.html/79dac47ea50b5dfbd3f234e368ed61d247fb0eb989f87fe01aedaf25@%3Cdev.calcite.apache.org%3E
>
> Best,
> Danny Chan
> 在 2019年10月31日 +0800 PM3:56,Vladimir Ozerov ,写道:
> > Hi colleagues,
> >
> > I would like to discuss with the community the possibility of adding a
> new
> > public method to VolcanoPlanner which will forcefully re-trigger the
> rules
> > for the specific rel. This is a follow up of a discussion started in the
> > other thread [1].
> >
> > **Problem statement**
> > When converting between conventions during optimization VolcanoPlanner
> > prefers the top-bottom approach, so that the nodes are converted from the
> > root. But in some cases, the intermediate node must be converted after
> its
> > children. This is especially true for distributed SQL engines, which rely
> > on distribution traits during the optimization process: it is not
> possible
> > to efficiently choose a proper physical implementation of a parent node
> > unless the physical representation of a child node is known.
> >
> > It seems that presently VolcanoPlanner cannot address such cases by
> > default. Consider that we have two nodes and associated rules which
> convert
> > them to physical counterparts:
> > [LogicalParent <- LogicalChild]
> > The parent should be converted after the child. When the child is
> > converted, the physical node is created:
> > [LogicalParent <- {LogicalChild, PhysicalChild}]
> > In order to finish the optimization process, we need to convert the
> parent.
> > But parent rules are not fired, because PhysicalChild has traits
> > incompatible with LogicalParent.
> >
> > Presently the problem could be solved in two ways:
> > 1) Always produce conversions when going top-down. In this case, I first
> > create a physical parent, then a physical child. The problem is that
> > created parent is not optimal because it didn't know child distribution
> at
> > the time of creation. So the full flow would be: create a bad parent,
> > create a good child, create a good parent.
> > 1.1) [LogicalParent <- LogicalChild]
> > 1.2) [{LogicalParent, PhysicalParentBad} <- LogicalChild]
> > 1.3) [{LogicalParent, PhysicalParentBad} <- {LogicalChild,
> PhysicalChild}]
> > 1.4) [{LogicalParent, PhysicalParentBad, PhysicalParentGood} <-
> > {LogicalChild, PhysicalChild}]
> > What is worse, the creation of a not optimal parent will trigger rules on
> > its parent, which in turn may create a not optimal parent-parent node,
> etc.
> >
> > 2) Make sure that your convention returns true for
> > Convention.canConvertConvention. In this case, 

Re: [DISCUSS] Proposal to add API to force rules matching specific rels

2019-10-31 Thread Vladimir Ozerov
Hi Xiening,

Yes, I think that the manual creation of converters to trigger parent rules
should be enough at the moment. I'll try to explore this possibility. Thank
you.

Regards,
Vladimir

чт, 31 окт. 2019 г. в 20:11, Xiening Dai :

> Hi Vladimir,
>
> I think for short/mid term, #2 way (using AbstractConverter) should work
> after we fix CALCITE-2970. We already understand the root cause, now are
> looking at the best way to fix it. If you cannot wait, you can also create
> your own converter rule so it won’t generate logical node, and the
> performance should be much better. And to your concern regarding the
> overhead of creating AbstractConverter objects, I think they are just minor
> overheads compared to the rest of part of the work framework’s doing (rule
> matching, merging set, etc).
>
> Regarding the comment "to explore potential traits of children before
> optimizing parents”, I totally agree and want to point out we should also
> consider parent requirements. Currently such mechanism is missing in
> Calcite, and we are having discussion on how to add it as part of the
> Volcano planner. Danny'd shared the mail archive previously. Welcome to
> join the discussion.
>
>
> > On Oct 31, 2019, at 3:37 AM, Vladimir Ozerov  wrote:
> >
> > Hi Danny,
> >
> > Thank you very much for the links. What is described here is pretty much
> > similar to the problem I describe. Especially the discussion about trait
> > propagation, as this is basically what I need - to explore potential
> traits
> > of children before optimizing parents. And this is basically what Drill
> > already does with it's SubsetTransformer:
> > 1) There is a SubsetTransformer interface, which iterates over physical
> > relations of the given subset [1]
> > 2) If you want to make a physical project, you iterate over physical
> > relations of the input subset and create possible physical projects [2]
> > 3) But if you cannot find any physical input, then you trigger creation
> of
> > a "bad" physical project, which is very likely to have poor cost because
> it
> > cannot take advantage of input's distribution information [3]
> > So, step 2 - is a trait set propagation which is needed by many
> > distributed engines. Step 3 - an attempt to workaround current
> > VolcanoPlanner behavior, when a parent rule is fired only if produced
> child
> > node has compatible trait set.
> >
> > I do not know Calcite's architecture that good but at first glance, the
> > proposed ability to re-fire rules of a specific Rel seems good enough. It
> > doesn't expand search space, because no new nodes are created, and it
> seems
> > to be relatively simple to implement.
> >
> > [1]
> >
> https://github.com/apache/drill/blob/1.16.0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
> > [2]
> >
> https://github.com/apache/drill/blob/1.16.0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java#L66
> > [3]
> >
> https://github.com/apache/drill/blob/1.16.0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java#L69
> >
> > чт, 31 окт. 2019 г. в 12:21, Danny Chan :
> >
> >> Thanks Vladimir for bringing up this discussion !
> >>
> >> Did you notice that there is a JIRA issue about this problem ? [1] Also
> a
> >> discussion about how to propagate the traits [2]
> >>
> >> [1] https://issues.apache.org/jira/browse/CALCITE-2970
> >> [2]
> >>
> https://ponymail-vm.apache.org/_GUI_/thread.html/79dac47ea50b5dfbd3f234e368ed61d247fb0eb989f87fe01aedaf25@%3Cdev.calcite.apache.org%3E
> >>
> >> Best,
> >> Danny Chan
> >> 在 2019年10月31日 +0800 PM3:56,Vladimir Ozerov ,写道:
> >>> Hi colleagues,
> >>>
> >>> I would like to discuss with the community the possibility of adding a
> >> new
> >>> public method to VolcanoPlanner which will forcefully re-trigger the
> >> rules
> >>> for the specific rel. This is a follow up of a discussion started in
> the
> >>> other thread [1].
> >>>
> >>> **Problem statement**
> >>> When converting between conventions during optimization VolcanoPlanner
> >>> prefers the top-bottom approach, so that the nodes are converted from
> the
> >>> root. But in some cases, the intermediate node must be converted after
> >> its
> >>> children. This is especially true for distributed SQL engines, which
> rely
> >>> on distribution traits 

Re: Problem with converters and possibly rule matching

2019-11-01 Thread Vladimir Ozerov
t.
>
> *Part II: Example with the original Volcano optimizer*
>
> Now let me try to show how the original (from the paper) Volcano optimizer
> could handle your query.
>
> The optimization starts and we request two properties convention and
> distribution.
>
> RootLogicalRel[convention=PHYSICAL, distribution=SINGLETON]
> -> ProjectLogicalRel
>   -> MapScanLogicalRel
>
> Rule 1: RootLogicalRel -> RootPhysicalRel applies the algorithm but cannot
> satisfy the SINGLETON property so it propagates the demand
>
> RootPhysicalRel
> -> ProjectLogicalRel [convention=PHYSICAL, distribution=SINGLETON]
>   -> MapScanLogicalRel
>
> Rule 2: ProjectLogicalRel -> ProjectPhysicalRel applies the algorithm but
> cannot satisfy the SINGLETON property so it propagates the demand
>
> RootPhysicalRel
> -> ProjectPhysicalRel
>   -> MapScanLogicalRel [convention=PHYSICAL, distribution=SINGLETON]
>
> Rule 3: MapScanLogicalRel -> MapScanPhysicalRel applies the algorithm and
> given that the table is replicated it satisfies the distribution property.
>
> RootPhysicalRel
> -> ProjectPhysicalRel
>   -> MapScanPhysicalRel
>
> So we end up with a complete plan that satisfies all properties and does
> not have redundant exchange operators.
> Moreover we didn't require all possible distributions when we applied Rule
> 2 since we just want to satisfy the SINGLETON property requested by the
> parent.
> The example above does not demonstrate the additional optimization paths
> which would be apply an enforcer (an Exchange operator to satisfy the
> SINGLETON property) at a higher level than the scan.
>
> Would this be an acceptable approach? Does it still suffer from a big
> search space?
>
> To make the analog with the actual Volcano optimizer in Calcite the thing
> that may be missing is to be able to know what trait was requested by the
> parent during the application of Rule 2.
>
> Best,
> Stamatis
>
> [1]
>
> https://lists.apache.org/thread.html/79dac47ea50b5dfbd3f234e368ed61d247fb0eb989f87fe01aedaf25@%3Cdev.calcite.apache.org%3E
>
> On Wed, Oct 30, 2019 at 7:24 PM Vladimir Ozerov 
> wrote:
>
> > Hi Stamatis,
> >
> > The problem that the presented reproducer is a very simplified version of
> > what is actually needed. In reality, there are several distribution
> types -
> > PARTITIONED, REPLICATED, SINGLETON. To make things worse, PARTITIONED may
> > or may not have concrete distribution fields. In theory, I can create one
> > transformation per distribution type, but that would increase plan space
> > significantly. In my sample "Root <- Project <- Scan" plan, there is no
> > room for optimization at all, we only need to convert the nodes and
> > propagate traits. But if I follow the proposed approach, the planner
> would
> > create three equivalent paths. For complex queries, this may increase
> > optimization time significantly.
> >
> > What I need instead is to gradually convert and optimize nodes
> *bottom-up*,
> > instead of top-bottom. That is, create a physical scan first, then
> create a
> > physical project on top of it, etc. But at the same time, some rules
> still
> > require the top-bottom approach. So essentially I need the optimizer to
> do
> > both. Abstract converters help me establish bottom-up preparation but do
> > this at the cost of considering too many trait pairs, and as a result,
> also
> > pollute the search space.
> >
> > To the contrast, precise command "I transformed the node A to node B,
> > please re-trigger the rules for A's parents" would allow us to re-trigger
> > only required rules, without adding more nodes.
> >
> > Does it make sense?
> >
> > Regards,
> > Vladimir.
> >
> > ср, 30 окт. 2019 г. в 21:02, Stamatis Zampetakis :
> >
> > > I admit that I didn't thoroughly read the exchanges so far but forcing
> > the
> > > rules trigger does not seem the right approach.
> > >
> > > Other than that I would like to backtrack a bit to point 4.3 raised by
> > > Vladimir.
> > >
> > > "ProjectPhysicalRule [6] - transforms logical project to physical
> > > project *ONLY* if there is an underlying physical input with REPLICATED
> > or
> > > SINGLETON distribution"
> > >
> > > The rule could be modified to do the following two transformations:
> > > 1. Create a physical project and require the input to be REPLICATED.
> > > 2. Create a physical project and require
> > > the input to be SINGLETON.
> > >
> > &

Re: Re: Problem with converters and possibly rule matching

2019-11-05 Thread Vladimir Ozerov
Hi Haisheng,

 think I already tried something very similar to what you explained, but it
gave not an optimal plan. Please let me describe what I did. I would
appreciate your feedback.

1) We start with a simple operator tree Root <- Project <- Scan, where the
root is a final aggregator in the distributed query engine:
-> LogicalRoot
 -> LogicalProject
  -> LogicalScan

2) First, we convert the Root and enforce SINGLETON distribution on a child:
*-> PhysicalRoot[SINGLETON]*
* -> Enforcer#1[SINGLETON]*
  -> LogicalProject
   -> LogicalScan

3) Then the project's rule is invoked. It doesn't know the distribution of
the input, so it requests ANY distribution. Note that we have to set ANY to
the project as well since we do not know the distribution of the input:
-> PhysicalRoot[SINGLETON]
 -> Enforcer#1[SINGLETON]
*  -> PhysicalProject[ANY]*
*   -> Enforcer#2[ANY]*
-> LogicalScan

4) Finally, the physical scan is created and its distribution is resolved.
Suppose that it is REPLICATED, i.e. the whole result set is located on all
nodes.
-> PhysicalRoot[SINGLETON]
 -> Enforcer#1[SINGLETON]
  -> PhysicalProject[ANY]
   -> Enforcer#2[ANY]
*-> PhysicalScan[REPLICATED]*

5) Now as all logical nodes are converted, we start resolving enforcers.
The second one is no-op, since REPLICATED satisfies ANY:
-> PhysicalRoot[SINGLETON]
 -> Enforcer#1[SINGLETON]
  -> PhysicalProject[ANY]
   -> PhysicalScan[REPLICATED]

6) But the first enforcer now requires an Exchange, since ANY doesn't
satisfy SINGLETON!
-> PhysicalRoot[SINGLETON]
* -> SingletonExchange[SINGLETON]*
  -> PhysicalProject[ANY] // <= unresolved!
   -> PhysicalScan[REPLICATED]

The resulting plan requires data movement only because we didn't know
precise distribution of the PhysicalProject when it was created. But should
I enable Convention.Impl.canConvertConvention, bottom-up propagation kicks
in, and the correct plan is produced because now LogicalProject has a
chance to be converted to PhysicalProject with the concrete distribution.
The optimized plan looks like this (since REPLICATED satisfies SINGLETON):
-> PhysicalRoot[SINGLETON]
 -> PhysicalProject[REPLICATED]
  -> PhysicalScan[REPLICATED]

You may see this in action in my reproducer:
1) Test producing "bad" plan:
https://github.com/devozerov/calcite-optimizer/blob/master/src/test/java/devozerov/OptimizerTest.java#L45
2) Root enforces SINGLETON on Project:
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/RootPhysicalRule.java#L45
3) Project enforces default (ANY) distribution on Scan:
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/ProjectPhysicalRule.java#L49

Please let me know if this flow is similar to what you meant.

Regards,
Vladimir.

пн, 4 нояб. 2019 г. в 10:33, Haisheng Yuan :

> Hi Vladimir,
>
> This is still can be done through top-down request approach.
>
> PhysicalFilter operator should request ANY distribution from child
> operator, unless there is outer reference in the filter condition, in which
> case, PhysicalFilter should request SINGLETON or BROADCAST distribution. So
> in your case, PhysicalFilter request ANY, its required distribution will be
> enforced on filter's output.
>
> Regarding index usage, you should have a FIlterTableScan2IndexGet logical
> transformation rule, and a IndexGet2IndexScan physical implementation rule.
> Note that IndexGet is a logical operator and IndexScan is a physical
> operator, which are also used by SQL Server.
>
> - Haisheng
>
> --
> 发件人:Vladimir Ozerov
> 日 期:2019年11月01日 17:30:26
> 收件人:
> 主 题:Re: Problem with converters and possibly rule matching
>
> Hi Stamatis,
>
> Thank you for your reply. I also thought that we may set the distribution
> trait during logical planning because it is known in advance. And the
> example I gave will work! :-) But unfortunately, it will work only because
> the tree is very simple, and the Project is adjacent to the Scan. This is
> how my reproducer will work in that case:
> 1) Root: enforce "SINGLETON" on Project
> 2) Project: check the logical Scan, infer already resolved distribution,
> then convert to [PhyiscalProject <- PhysicalScan]
> 3) Resolve Root enforcer, adding and Exchange if needed.
>
> But this stops working as soon as a plan becomes more complex so that it is
> impossible to infer the distribution from the child immediately. E.g.:
> LogicalRoot [distribution=SINGLETON]
> -> LogicalProject // We are here new and cannot produce the physical
> project
> -> LogicalFilter[distribution=?]
> -> LogicalScan[distribution=REPLICATED]
>
> This is where your suggestion with cascading e

Re: [DISCUSS] Proposal to add API to force rules matching specific rels

2019-11-05 Thread Vladimir Ozerov
Hi Xiening,

I read the thread about on-demand trait requests. It seems pretty similar
to what I am trying to achieve, as it facilitates the bottom-up propagation
of physical traits. In fact, both your and my strategy propagate traits
bottom-up, but I do this through rules, which also fire bottom-up, while in
your case only the traits are propagated bottom-up, while rules continue
working in a top-down fashion.

However, I am thinking of how I would potentially implement my optimizer
with your approach, and it feels like with on-demand traits resulting
implementation of metadata queries may become very complex to that point
that it will look like another set of rules, parallel to the already
existing ruleset. For example, consider that I have a couple of distributed
tables in an OLTP application. These tables have a number of indexes, and I
would like to join them. First, I have a number of choices on how to join
tables with respect to distribution. Then, I have a number of choices on
which access method to use. Because sometimes it is beneficial to pick
index scans instead of table scans even without index conditions, for
example, to preserve a comfortable collation. So when my logical scan
receives such metadata request, it typically cannot return all possible
combinations, because there are too many of them. Instead, some heuristical
or cost-based logic will be used to calculate a couple of most prospective
ones. But it seems that we will have to duplicate the same logic in the
corresponding rule, aren't we?

I would love to read your design because this is a really interesting
topic, and it is of great importance for the distributed engines developed
on top of Calcite since proper use of distribution and collation is the key
success factor for efficient query optimization.

Regards,
Vladimir.

пт, 1 нояб. 2019 г. в 00:40, Xiening Dai :

> Actually we solved this problem in our setup using a mechanism called
> “Pull-Up Traits”, which explores the possible trait set of children’s input
> to decide parent’s physical properties. In order to determine child input
> trait, you would have to look at child’s children, and all the way to the
> leaves nodes or a barrier. A barrier is a rel node which cannot derive any
> traits regardless the input. A good example would be a user define function
> which would throw off any distribution or collation. Then we realize just
> pulling up is not enough, sometimes we would need to look at parent’s
> requirement as well. So we try to solve this in a unified framework, which
> we call “On Demand Trait” and implement it as part of the framework so
> anyone can be benefited. I hope Haisheng can share a design doc once we
> have more concrete ideas.
>
>
> > On Oct 31, 2019, at 11:37 AM, Jinfeng Ni  wrote:
> >
> > Hi Vladimir,
> >
> > The SubsetTransformer interface and the iterating over the RelNodes
> > within a RelSubset in Drill  is exactly implemented to do the trait
> > propagation. We also had to rely on AbstractConverter to fire
> > necessary rule to avoid the CanNotPlan issue. At some point, Calcite
> > community chooses to remove AbstractConverter and Drill had to add it
> > back, which is probably one of the main reasons for us to continue
> > using a Calcite fork.  I still remember we constantly had to deal with
> > the dilemma between "CanNotPlan" and long planing time due to explored
> > search space.
> >
> > Glad to see more people are joining the effort to solve this long
> > overdue issue, something missing in Calcite's core optimizer framework
> > "since before Calcite was Calcite" (Jacques's words).
> >
> > Jinfeng
> >
> >
> > On Thu, Oct 31, 2019 at 3:38 AM Vladimir Ozerov 
> wrote:
> >>
> >> Hi Danny,
> >>
> >> Thank you very much for the links. What is described here is pretty much
> >> similar to the problem I describe. Especially the discussion about trait
> >> propagation, as this is basically what I need - to explore potential
> traits
> >> of children before optimizing parents. And this is basically what Drill
> >> already does with it's SubsetTransformer:
> >> 1) There is a SubsetTransformer interface, which iterates over physical
> >> relations of the given subset [1]
> >> 2) If you want to make a physical project, you iterate over physical
> >> relations of the input subset and create possible physical projects [2]
> >> 3) But if you cannot find any physical input, then you trigger creation
> of
> >> a "bad" physical project, which is very likely to have poor cost
> because it
> >> cannot take advantage of input's distribution information [3]
> >> So, step 2 - is

Re: Re: Re: Problem with converters and possibly rule matching

2019-11-14 Thread Vladimir Ozerov
Hi Haisheng,

I double-checked the code. My original version returned false for some
cases, but it didn't affect number of rules calls anyway, so I changed it
to always return true. Please note that if I change the code as you
suggested, the test started failing, because bottom-up propagation of rule
calls no longer work: when the child is converted to physical form, the
parent logical node is not notified. This is the very problem I address
with that weird physical-to-logical conversions: they do not make sense,
and converter expansion does not produce any new rels, but their existence
allow for logical rule re-trigger which ultimately allow the plan to
compile.

Regarding two conventions - I agree that it may look strange, but I do not
see any problems from the correctness perspective. Separation of logical
and physical planning helps me avoid excessive expansion of the search
place: I do not want my physical rules to produce new rels from not
optimized logical rels. Do you see any problems with that approach?

Regards,
Vladimir

ср, 6 нояб. 2019 г. в 03:52, Haisheng Yuan :

> Hi Vladimir,
>
> The code in PHYSICAL convention L44 looks weird, I think it always returns
> true.
>
> https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/HazelcastConventions.java#L44
>
> Try this:
>
> fromTraits.containsIfApplicable(Convention.PHYSICAL)
> && toTraits.containsIfApplicable(Convention.PHYSICAL);
>
>
> Adding a AbstractConverter on logical operators is meaningless. Calcite is
> mixing the concept of logical and physical together, which is sad.
>
> BTW, using 2 conventions is not appropriate and wrong.
>
> - Haisheng
>
> ------
> 发件人:Vladimir Ozerov
> 日 期:2019年11月05日 18:02:15
> 收件人:Haisheng Yuan
> 抄 送:dev@calcite.apache.org (dev@calcite.apache.org) >
> 主 题:Re: Re: Problem with converters and possibly rule matching
>
> Hi Haisheng,
>
>  think I already tried something very similar to what you explained, but
> it gave not an optimal plan. Please let me describe what I did. I would
> appreciate your feedback.
>
> 1) We start with a simple operator tree Root <- Project <- Scan, where
> the root is a final aggregator in the distributed query engine:
> -> LogicalRoot
>  -> LogicalProject
>   -> LogicalScan
>
> 2) First, we convert the Root and enforce SINGLETON distribution on a
> child:
> *-> PhysicalRoot[SINGLETON]*
> * -> Enforcer#1[SINGLETON]*
>   -> LogicalProject
>-> LogicalScan
>
> 3) Then the project's rule is invoked. It doesn't know the distribution of
> the input, so it requests ANY distribution. Note that we have to set ANY to
> the project as well since we do not know the distribution of the input:
> -> PhysicalRoot[SINGLETON]
>  -> Enforcer#1[SINGLETON]
> *  -> PhysicalProject[ANY]*
> *   -> Enforcer#2[ANY]*
> -> LogicalScan
>
> 4) Finally, the physical scan is created and its distribution is resolved.
> Suppose that it is REPLICATED, i.e. the whole result set is located on all
> nodes.
> -> PhysicalRoot[SINGLETON]
>  -> Enforcer#1[SINGLETON]
>   -> PhysicalProject[ANY]
>-> Enforcer#2[ANY]
> *-> PhysicalScan[REPLICATED]*
>
> 5) Now as all logical nodes are converted, we start resolving enforcers.
> The second one is no-op, since REPLICATED satisfies ANY:
> -> PhysicalRoot[SINGLETON]
>  -> Enforcer#1[SINGLETON]
>   -> PhysicalProject[ANY]
>-> PhysicalScan[REPLICATED]
>
> 6) But the first enforcer now requires an Exchange, since ANY doesn't
> satisfy SINGLETON!
> -> PhysicalRoot[SINGLETON]
> * -> SingletonExchange[SINGLETON]*
>   -> PhysicalProject[ANY] // <= unresolved!
>-> PhysicalScan[REPLICATED]
>
> The resulting plan requires data movement only because we didn't know
> precise distribution of the PhysicalProject when it was created. But should
> I enable Convention.Impl.canConvertConvention, bottom-up propagation
> kicks in, and the correct plan is produced because now LogicalProject has
> a chance to be converted to PhysicalProject with the concrete
> distribution. The optimized plan looks like this (since REPLICATED
> satisfies SINGLETON):
> -> PhysicalRoot[SINGLETON]
>  -> PhysicalProject[REPLICATED]
>   -> PhysicalScan[REPLICATED]
>
> You may see this in action in my reproducer:
> 1) Test producing "bad" plan:
> https://github.com/devozerov/calcite-optimizer/blob/master/src/test/java/devozerov/OptimizerTest.java#L45
> 2) Root enforces SINGLETON on Project:
> https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/RootPhysi

Re: Re: Re: Problem with converters and possibly rule matching

2019-11-14 Thread Vladimir Ozerov
search place* = search space

чт, 14 нояб. 2019 г. в 13:10, Vladimir Ozerov :

> Hi Haisheng,
>
> I double-checked the code. My original version returned false for some
> cases, but it didn't affect number of rules calls anyway, so I changed it
> to always return true. Please note that if I change the code as you
> suggested, the test started failing, because bottom-up propagation of rule
> calls no longer work: when the child is converted to physical form, the
> parent logical node is not notified. This is the very problem I address
> with that weird physical-to-logical conversions: they do not make sense,
> and converter expansion does not produce any new rels, but their existence
> allow for logical rule re-trigger which ultimately allow the plan to
> compile.
>
> Regarding two conventions - I agree that it may look strange, but I do not
> see any problems from the correctness perspective. Separation of logical
> and physical planning helps me avoid excessive expansion of the search
> place: I do not want my physical rules to produce new rels from not
> optimized logical rels. Do you see any problems with that approach?
>
> Regards,
> Vladimir
>
> ср, 6 нояб. 2019 г. в 03:52, Haisheng Yuan :
>
>> Hi Vladimir,
>>
>> The code in PHYSICAL convention L44 looks weird, I think it always
>> returns true.
>>
>> https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/HazelcastConventions.java#L44
>>
>> Try this:
>>
>> fromTraits.containsIfApplicable(Convention.PHYSICAL)
>> && toTraits.containsIfApplicable(Convention.PHYSICAL);
>>
>>
>> Adding a AbstractConverter on logical operators is meaningless. Calcite
>> is mixing the concept of logical and physical together, which is sad.
>>
>> BTW, using 2 conventions is not appropriate and wrong.
>>
>> - Haisheng
>>
>> --
>> 发件人:Vladimir Ozerov
>> 日 期:2019年11月05日 18:02:15
>> 收件人:Haisheng Yuan
>> 抄 送:dev@calcite.apache.org (dev@calcite.apache.org)<
>> dev@calcite.apache.org>
>> 主 题:Re: Re: Problem with converters and possibly rule matching
>>
>> Hi Haisheng,
>>
>>  think I already tried something very similar to what you explained, but
>> it gave not an optimal plan. Please let me describe what I did. I would
>> appreciate your feedback.
>>
>> 1) We start with a simple operator tree Root <- Project <- Scan, where
>> the root is a final aggregator in the distributed query engine:
>> -> LogicalRoot
>>  -> LogicalProject
>>   -> LogicalScan
>>
>> 2) First, we convert the Root and enforce SINGLETON distribution on a
>> child:
>> *-> PhysicalRoot[SINGLETON]*
>> * -> Enforcer#1[SINGLETON]*
>>   -> LogicalProject
>>-> LogicalScan
>>
>> 3) Then the project's rule is invoked. It doesn't know the distribution
>> of the input, so it requests ANY distribution. Note that we have to set ANY
>> to the project as well since we do not know the distribution of the input:
>> -> PhysicalRoot[SINGLETON]
>>  -> Enforcer#1[SINGLETON]
>> *  -> PhysicalProject[ANY]*
>> *   -> Enforcer#2[ANY]*
>> -> LogicalScan
>>
>> 4) Finally, the physical scan is created and its distribution is
>> resolved. Suppose that it is REPLICATED, i.e. the whole result set is
>> located on all nodes.
>> -> PhysicalRoot[SINGLETON]
>>  -> Enforcer#1[SINGLETON]
>>   -> PhysicalProject[ANY]
>>-> Enforcer#2[ANY]
>> *-> PhysicalScan[REPLICATED]*
>>
>> 5) Now as all logical nodes are converted, we start resolving enforcers.
>> The second one is no-op, since REPLICATED satisfies ANY:
>> -> PhysicalRoot[SINGLETON]
>>  -> Enforcer#1[SINGLETON]
>>   -> PhysicalProject[ANY]
>>-> PhysicalScan[REPLICATED]
>>
>> 6) But the first enforcer now requires an Exchange, since ANY doesn't
>> satisfy SINGLETON!
>> -> PhysicalRoot[SINGLETON]
>> * -> SingletonExchange[SINGLETON]*
>>   -> PhysicalProject[ANY] // <= unresolved!
>>-> PhysicalScan[REPLICATED]
>>
>> The resulting plan requires data movement only because we didn't know
>> precise distribution of the PhysicalProject when it was created. But should
>> I enable Convention.Impl.canConvertConvention, bottom-up propagation
>> kicks in, and the correct plan is produced because now LogicalProject
>> has a chance to be converted to PhysicalProject with the concrete
>

Re: [DISCUSS] Proposal to add API to force rules matching specific rels

2019-11-14 Thread Vladimir Ozerov
Hi Xing,

Thanks for your suggestion. Yes, this may help, and if I get your idea
right, I already had it in my reproducer:
1) Create the converted physical input:
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/ProjectPhysicalRule.java#L49
2) Use it in case no physical children were found:
https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/ProjectPhysicalRule.java#L79

This idea is borrowed from Apache Drill physical rules. But the problem is
that this approach leads to a suboptimal plan - parent node doesn't know
the future distribution of a child node. And as a result, it doesn't know
it's own distribution. So the final plan is constructed in that way:
1.1) Root enforced "SINGLETON" on its child:
-> PhysicalRoot[SINGLETON]
 -> Converter[SINGLETON]
  -> PhysicalProject[*ANY*]
   -> PhysicalScan[REPLICATED]

1.2) But since the child (PhysicalProject) failed to infer distribution
during rule call, it falls back to ANY distribution. In order to satisfy
SINGLETON distribution of a parent, we inject an exchange in the final plan:
-> PhysicalRoot[SINGLETON]
* -> Exchange[SINGLETON]*
  -> PhysicalProject[*ANY*]
   -> PhysicalScan[REPLICATED]

2) But this is a suboptimal plan. The optimal plan is:
-> PhysicalRoot[SINGLETON]
 -> PhysicalProject[REPLICATED]
  -> PhysicalScan[REPLICATED]

You may observe it in my tests:
1)
https://github.com/devozerov/calcite-optimizer/blob/master/src/test/java/devozerov/OptimizerTest.java#L46
-
works as you described and produces not optimal plan with exchange
2)
https://github.com/devozerov/calcite-optimizer/blob/master/src/test/java/devozerov/OptimizerTest.java#L30
-
rely on AbstractConverter-s and produce an optimal plan with bottom-up
trait propagation at the cost of significantly increased planning time

Regards,
Vladimir.

пт, 8 нояб. 2019 г. в 16:15, XING JIN :

> Hi Vladimir,
>
> I think the way PlannerTests#GoodSingleRule and EnumerableXXXRule work may
> help you ~
> They work by a top-down fashion, but when matching parent, they convert the
> children explicitly.
> You may try below steps:
> 1. Construct a rule LogicalParentRule to match the LogicalParent without
> distribution/physical requirement for the LogicalChild;
> 2. In this rule, you call 'planner.changeTraits' on the LogicalChild to
> build a new child with physical convention. Note that at this moment only
> an empty RelSubset is created and no PhysicalChild exists.
> 3. Then set the RelNode to be the new input of LogicalParent;
>
> By above steps, you can build a parent-child relationship between
> LogicalParent and PhysicalChild, and at last the PhysicalParentRule will be
> fired based on this relationship.
>
> I have a commit to illustrate my idea, check VolcanoPlannerTest#testDEV in
> below link, hope it may help you ~
> https://github.com/jinxing64/calcite/tree/demo
>
> Also I'm +1 with Seliverstov that to get all parents of a set, which
> against the current check in RelSubset#getParentRels
>
> Best,
> Jin
>
> Vladimir Ozerov  于2019年11月5日周二 下午6:41写道:
>
> > Hi Xiening,
> >
> > I read the thread about on-demand trait requests. It seems pretty similar
> > to what I am trying to achieve, as it facilitates the bottom-up
> propagation
> > of physical traits. In fact, both your and my strategy propagate traits
> > bottom-up, but I do this through rules, which also fire bottom-up, while
> in
> > your case only the traits are propagated bottom-up, while rules continue
> > working in a top-down fashion.
> >
> > However, I am thinking of how I would potentially implement my optimizer
> > with your approach, and it feels like with on-demand traits resulting
> > implementation of metadata queries may become very complex to that point
> > that it will look like another set of rules, parallel to the already
> > existing ruleset. For example, consider that I have a couple of
> distributed
> > tables in an OLTP application. These tables have a number of indexes,
> and I
> > would like to join them. First, I have a number of choices on how to join
> > tables with respect to distribution. Then, I have a number of choices on
> > which access method to use. Because sometimes it is beneficial to pick
> > index scans instead of table scans even without index conditions, for
> > example, to preserve a comfortable collation. So when my logical scan
> > receives such metadata request, it typically cannot return all possible
> > combinations, because there are too many of them. Instead, some
> heuristical
> > or cost-based logic will be used to calculate a couple of most
> prospective
> > ones. But it seems that we will ha

Re: [DISCUSS] Support Sql Hint for Calcite

2019-11-14 Thread Vladimir Ozerov
Hi Danny,

Thank you very much for making this happen. Query hints are a very valuable
addition to the product.

Regards,
Vladimir.

вс, 10 нояб. 2019 г. в 05:58, Danny Chan :

> Hi, fellows, I’m planning to merge the hints PR in the following week, I’m
> very appreciated if you have other more review comment address. [1]
>
> Or if you have other thoughts, please address it here :)
>
> [1] https://github.com/apache/calcite/pull/1354
>
> Best,
> Danny Chan
> 在 2019年10月30日 +0800 AM1:42,Julian Hyde ,写道:
> > Sure, we can make sure something gets into 1.22. There is consensus
> about the parser extensions, whereas the extensions to RelNode and the
> planner engine are a little more experimental. So let’s go forward with
> that, stating which parts we think are likely to change.
> >
> > Julian
> >
> >
> > > On Oct 29, 2019, at 2:09 AM, Seliverstov Igor 
> wrote:
> > >
> > > Colleagues,
> > >
> > > Not only Hazelcast and Apache Flink are interested in SQL hints.
> Apache Ignite community is working on Calcite integration too, it’s
> important for us to have appropriate API at current development stage. This
> case we’ll be able to adapt our solution for SQL hints usage, probably
> determining additional approach weaknesses or inconveniences.
> > >
> > > Regards,
> > > Igor
> > >
> > > > 29 окт. 2019 г., в 11:51, Danny Chan 
> написал(а):
> > > >
> > > > Julian, can we make some effort to push this feature into release
> 1.22, there are users like Vladimir Ozerov from Hazelcast that are
> interesting on this feature, also the Apache Flink.
> > > >
> > > > I agree that this internal design is not that perfect, at this
> moment, we may hardly to conclude a perfect solution, but at least, the
> syntax would remain unchanged in the future.
> > > >
> > > > So can we mark this feature as experimental and we can promote the
> internal design when accept more feedbacks from the Calcite uses (from
> Apache Flink or from users like Vladimir).
> > > >
> > > > Best,
> > > > Danny Chan
> > > > 在 2019年10月18日 +0800 AM4:55,Julian Hyde ,写道:
> > > > > I wonder whether it is possible to add some kind of “action
> handler” to the planner engine, called, for example, when a rule has fired
> and is registering the RelNode created by the rule. People can write their
> own action handlers to copy hints around. Since the action handlers are the
> user’s code, they can iterate faster to find a hint-propagation strategy
> that works in practice.
> > > > >
> > > > > Another idea is to use VolcanoPlanner.Provenance[1]. A RelNode can
> find its ancestor RelNodes, and the rules that fired to create it. So it
> can grab hints from those ancestors. It does not need to copy those hints
> onto itself.
> > > > >
> > > > > Julian
> > > > >
> > > > > [1]
> https://calcite.apache.org/apidocs/org/apache/calcite/plan/volcano/VolcanoPlanner.Provenance.html
> <
> https://calcite.apache.org/apidocs/org/apache/calcite/plan/volcano/VolcanoPlanner.Provenance.html
> >
> > > > >
> > > > > > On Oct 16, 2019, at 8:38 PM, Haisheng Yuan <
> h.y...@alibaba-inc.com> wrote:
> > > > > >
> > > > > > Julian,
> > > > > > Your concern is very valid, and that is also our main concern.
> > > > > > I was thinking whether we can put hint into the MEMO group, so
> that both logical and physical expression in the same group can share the
> same hint, without copying the hint explicitly. But for newly generated
> expression that doesn't belong to the original group, we still need to copy
> hints. What's worse, in HepPlanner, there is no such concept, we may still
> need to copy hints explicity in planner rules, if we want to keep the hint,
> which is burdensome.
> > > > > >
> > > > > > - Haisheng
> > > > > >
> > > > > >
> --
> > > > > > 发件人:Danny Chan
> > > > > > 日 期:2019年10月16日 14:54:46
> > > > > > 收件人:
> > > > > > 主 题:Re: [DISCUSS] Support Sql Hint for Calcite
> > > > > >
> > > > > > Thanks for the clarification.
> > > > > >
> > > > > > I understand you worried. Yes, the effort/memory would be wasted
> or meaningless if hints are not used. This is just what a hint does, it is
> a “hint” and non-man

Re: [DISCUSS] Proposal to add API to force rules matching specific rels

2019-11-19 Thread Vladimir Ozerov
HI Igor,

Thank you for the details. Meanwhile, I solved it with separation of
conversion rules from the physical optimization rules. So the first pass
creates physical nodes with unknown physical properties (top-bottom), while
subsequent processing of the leaf nodes triggers rules which convert "bad"
physical nodes to "good" physical nodes with know distribution and
collation.

Regards,
Vladimir.

пн, 18 нояб. 2019 г. в 13:43, Seliverstov Igor :

> Vladimir,
>
> Hope it may help you.
>
> Currently we applied the next way (just rough description):
>
> 1) We created an API to derive possible traits permutations on the basis
> of children traits  (quite similar to one, described in «On Demand trait
> set request» topic)
>
> 2) added a general rule that copies Logical nodes, but requesting our
> convention from their children (IGNITE convention, ANY distribution, EMPTY
> collation) and sets importance of old Logical nodes to zero - so, we have a
> Logical parent which input satisfies any possible distribution and no rules
> matched to previous logical node any more.
>
> 3) Physical rules to create physical rel nodes only if physical traits may
> be derived (there is no «barrier», described in one of previous messages) -
> derived traits are a collection, we don’t create a physical rel node for
> each possible traits set, also we may set zero importance for previously
> created rel nodes to decrease search space.
>
> Now we know actual and required distribution, we don’t need
> AbstractConverters and able just call TraitDef.convert() method inside a
> rule.
> A rule still able to produce the same output several times, but
> «memorization" inside the planner solves it for us.
>
> preliminary tests show almost zero overhead of the approach.
>
> Regards,
> Igor
>
>
> > 14 нояб. 2019 г., в 14:49, Vladimir Ozerov 
> написал(а):
> >
> > Hi Xing,
> >
> > Thanks for your suggestion. Yes, this may help, and if I get your idea
> > right, I already had it in my reproducer:
> > 1) Create the converted physical input:
> >
> https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/ProjectPhysicalRule.java#L49
> > 2) Use it in case no physical children were found:
> >
> https://github.com/devozerov/calcite-optimizer/blob/master/src/main/java/devozerov/physical/ProjectPhysicalRule.java#L79
> >
> > This idea is borrowed from Apache Drill physical rules. But the problem
> is
> > that this approach leads to a suboptimal plan - parent node doesn't know
> > the future distribution of a child node. And as a result, it doesn't know
> > it's own distribution. So the final plan is constructed in that way:
> > 1.1) Root enforced "SINGLETON" on its child:
> > -> PhysicalRoot[SINGLETON]
> > -> Converter[SINGLETON]
> >  -> PhysicalProject[*ANY*]
> >   -> PhysicalScan[REPLICATED]
> >
> > 1.2) But since the child (PhysicalProject) failed to infer distribution
> > during rule call, it falls back to ANY distribution. In order to satisfy
> > SINGLETON distribution of a parent, we inject an exchange in the final
> plan:
> > -> PhysicalRoot[SINGLETON]
> > * -> Exchange[SINGLETON]*
> >  -> PhysicalProject[*ANY*]
> >   -> PhysicalScan[REPLICATED]
> >
> > 2) But this is a suboptimal plan. The optimal plan is:
> > -> PhysicalRoot[SINGLETON]
> > -> PhysicalProject[REPLICATED]
> >  -> PhysicalScan[REPLICATED]
> >
> > You may observe it in my tests:
> > 1)
> >
> https://github.com/devozerov/calcite-optimizer/blob/master/src/test/java/devozerov/OptimizerTest.java#L46
> > -
> > works as you described and produces not optimal plan with exchange
> > 2)
> >
> https://github.com/devozerov/calcite-optimizer/blob/master/src/test/java/devozerov/OptimizerTest.java#L30
> > -
> > rely on AbstractConverter-s and produce an optimal plan with bottom-up
> > trait propagation at the cost of significantly increased planning time
> >
> > Regards,
> > Vladimir.
> >
> > пт, 8 нояб. 2019 г. в 16:15, XING JIN :
> >
> >> Hi Vladimir,
> >>
> >> I think the way PlannerTests#GoodSingleRule and EnumerableXXXRule work
> may
> >> help you ~
> >> They work by a top-down fashion, but when matching parent, they convert
> the
> >> children explicitly.
> >> You may try below steps:
> >> 1. Construct a rule LogicalParentRule to match the LogicalParent without
> >> distribution/physical requirement for the LogicalChild;
> >> 2. In this rule, you call 

Re: Re: Re: Problem with converters and possibly rule matching

2019-11-19 Thread Vladimir Ozerov
Hi Stamatis,

This approach worth trying, but I am afraid that it may produce too many
permutations. For example, this is the logical plan of one of TPC-H
benchmarks, and I am not very keen expanding it much further :-)

901:ProjectLogicalRel(ps_partkey=[$0], val=[$1])
  899:FilterLogicalRel(subset=[rel#900:Subset#66.LOGICAL.ANY.[]],
condition=[>($1, $2)])
897:JoinLogicalRel(subset=[rel#898:Subset#65.LOGICAL.ANY.[]],
condition=[true], joinType=[left])
  882:AggregateLogicalRel(subset=[rel#883:Subset#57.LOGICAL.ANY.[]],
group=[{0}], val=[SUM($1)])
880:ProjectLogicalRel(subset=[rel#881:Subset#56.LOGICAL.ANY.[]],
ps_partkey=[$1], $f1=[*($2, $3)])
  878:JoinLogicalRel(subset=[rel#879:Subset#55.LOGICAL.ANY.[]],
condition=[=($5, $6)], joinType=[inner])
875:JoinLogicalRel(subset=[rel#876:Subset#53.LOGICAL.ANY.[]],
condition=[=($0, $4)], joinType=[inner])

634:MapScanLogicalRel(subset=[rel#873:Subset#51.LOGICAL.ANY.[]],
table=[[partsupp]], projects=[[0, 1, 2, 3]])

632:MapScanLogicalRel(subset=[rel#874:Subset#52.LOGICAL.ANY.[]],
table=[[supplier]], projects=[[0, 1]])

630:MapScanLogicalRel(subset=[rel#877:Subset#54.LOGICAL.ANY.[]],
table=[[nation]], projects=[[0, 1]], filter=[=($1, '[NATION]')])
  895:AggregateLogicalRel(subset=[rel#896:Subset#64.LOGICAL.ANY.[]],
group=[{}], agg#0=[SINGLE_VALUE($0)])
893:ProjectLogicalRel(subset=[rel#894:Subset#63.LOGICAL.ANY.[]],
EXPR$0=[*($0, 0.9:DECIMAL(2, 1))])

891:AggregateLogicalRel(subset=[rel#892:Subset#62.LOGICAL.ANY.[]],
group=[{}], agg#0=[SUM($0)])

889:ProjectLogicalRel(subset=[rel#890:Subset#61.LOGICAL.ANY.[]], $f0=[*($1,
$2)])
  887:JoinLogicalRel(subset=[rel#888:Subset#60.LOGICAL.ANY.[]],
condition=[=($4, $5)], joinType=[inner])

885:JoinLogicalRel(subset=[rel#886:Subset#59.LOGICAL.ANY.[]],
condition=[=($0, $3)], joinType=[inner])

680:MapScanLogicalRel(subset=[rel#884:Subset#58.LOGICAL.ANY.[]],
table=[[partsupp]], projects=[[0, 2, 3]])

632:MapScanLogicalRel(subset=[rel#874:Subset#52.LOGICAL.ANY.[]],
table=[[supplier]], projects=[[0, 1]])

630:MapScanLogicalRel(subset=[rel#877:Subset#54.LOGICAL.ANY.[]],
table=[[nation]], projects=[[0, 1]], filter=[=($1, '[NATION]')])

I have two working solutions at the moment, which do not require
cross-convention converters:
1) Create separate rules for logical-physical node conversion, and for
physical-physical optimization
2) A kind of hack: add fake AbstractConverter with logical traits after the
physical node is created

I hope one of them will work good enough for the production case. Most
likely it would be p.1.

Regards,
Vladimir.


пт, 15 нояб. 2019 г. в 11:04, Stamatis Zampetakis :

> Hi again Vladimir,
>
> In my previous email, I mentioned three rules for performing logical to
> physical conversions.
> It's normal that if you add more operators we will end up with more rules.
> Now that the example has a filter we have the following rule list:
>
> Rule 1: RootLogicalRel -> RootPhysicalRel
> Rule 2: ProjectLogicalRel -> ProjectPhysicalRel
> Rule 3: FilterLogicalRel -> FilterPhysicalRel
> Rule 4: MapScanLogicalRel -> MapScanPhysicalRel
>
> If I understand well, your concern is that in order to get the optimal plan
> you would have to introduce many additional rules for the Exchange operator
> (e.g., ExchangeProjectTransposeRule, ExchangeFilterTransposeRue, etc.).
> I was thinking that it is not necessary to introduce these kind of rules.
> You could have only one rule, i.e., PhysicaExchangeRule, for enforcing a
> distribution that would be applied when there is a requirement for
> particular distribution.
>
> So for the initial plan
> LogicalRoot [distribution=SINGLETON]
> -> LogicalProject
>  -> LogicalFilter
>   -> LogicalScan
>
> The ruleset above should generate the following alternatives where the
> PhysicalExchangeRule is applied 3 times.
>
> Alt1:
> PhysicalRoot
> -> SingletonExchange
>  -> PhysicalProject
>   -> PhysicalFilter
>-> PhysicalScan
>
> Alt2:
> PhysicalRoot
> -> PhysicalProject
>  -> SingletonExchange
>   -> PhysicalFilter
>-> PhysicalScan
>
> Alt3:
> PhysicalRoot
> -> PhysicalProject
>  -> PhysicalFilter
>   -> SingletonExchange
>-> PhysicalScan
>
> This is still in the same spirit of the previous example where the physical
> property (distribution) is either satisfied immediately or passed down to
> the next level.
>
> Best,
> Stamatis
>
> On Thu, Nov 14, 2019 at 11:14 AM Vladimir Ozerov 
> wrote:
>
> > search place* = search space
> >
> > чт, 14 нояб. 2019 г. в 13:10, Vladimir Ozerov :
> >
> > > Hi Haisheng,
> > >
> > > I double-checked the code. My original version returned false for some
> > > cases, but it

Volcano's problem with trait propagation: current state and future

2019-12-05 Thread Vladimir Ozerov
Hi,

As I mentioned before, we are building a distributed SQL engine that uses
Apache Calcite for query optimization. The key problem we faced is the
inability to pull the physical traits of child relations efficiently. I'd
like to outline my understanding of the problem (I guess it was already
discussed multiple times) and ask the community to prove or disprove the
existence of that problem and its severity for the products which uses
Apache Calcite and ask for ideas on how it could be improved in the future.

I'll start with the simplified problem description and mentioned more
complex use cases then. Consider that we have a logical tree and a set of
implementation rules. Our goal is to find the optimal physical tree by
applying these rules. The classical Cascades-based approach directs the
optimization process from the top to the bottom (hence "top-down").
However, the actual implementation of tree nodes still happens bottom-up.
For the tree L1 <- L2, we enter "optimize(L1)", which recursively delegates
to "optimize(L2)". We then implement children nodes L1 <- [P2', P2''], and
return back to the parent, which is now able to pick promising
implementations of the children nodes and reject bad ones with the
branch-and-bound approach. AFAIK Pivotal's Orca works this way.

The Apache Calcite is very different because it doesn't allow the recursion
so that we lose the context on which node requested the child
transformation. This loss of context leads to the following problems:
1) The parent node cannot deduce it's physical properties during the
execution of the implementation rule, because Calcite expects the
transformation to be applied before children nodes are implemented. That is
if we are optimizing LogicalProject <- LogicalScan, we cannot set proper
distribution and collation for the to be created PhysicalProject, because
it depends on the distribution and collation of the children which is yet
to be resolved.
2) The branch-and-bound cannot be used because it requires at least one
fully-built physical subtree.

As a result of this limitation, products which rely on Apache Calcite for
query optimization, use one or several workarounds:

*1) Guess the physical properties of parent nodes before logical children
are implemented*
*Apache Flink* uses this strategy. The strategy is bad because of the
number of combinations of traits growth exponentially with the number of
attributes in the given RelNode, so you either explode the search space or
give up optimization opportunities. Consider the following tree:
LogicalSort[a ASC] <- LogicalFilter <- LogicalScan
The optimal implementation of the LogicalFilter is PhysicalFilter[collation=a
ASC] because it may eliminate the parent sort. But such optimization should
happen only if we know that there is a physical implementation of scan
allowing for this sort order, e.g. PhysicalIndexScan[collation=a ASC]. I.e.
we need to know the child physical properties first. Otherwise we fallback
to speculative approaches. With the *optimistic* approach, we emit all
possible combinations of physical properties, with the hope that the child
will satisfy some of them, thus expanding the search space exponentially.
With the *pessimistic* approach, we just miss this optimization opportunity
even if the index exists. Apache Flink uses the pessimistic approach.

*2) Use AbstractConverters*
*Apache Drill* uses this strategy. The idea is to "glue" logical and
physical operators, so that implementation of a physical child re-triggers
implementation rule of a logical parent. The flow is as follows:
- Invoke parent implementation rule - it either doesn't produce new
physical nodes or produce not optimized physical nodes (like in the Apache
Flink case)
- Invoke children implementation rules and create physical children
- Then converters kick-in and re-trigger parent implementation rule through
the creation of an abstract converter
- Finally, the parent implementation rule is fired again and now it
produces optimized node(s) since at least some of the physical
distributions of children nodes are implemented.

Note that this is essentially a hack to simulate the Cascades flow! The
problem is that AbstractConverters increase the complexity of planning
because they do not have any context, so parent rules are just re-triggered
blindly. Consider the optimization of the following tree:
LogicalJoin <- [LogicalScan1, LogicalScan2]
With the converter approach, the join implementation rule will be fired at
least 3 times, while in reality, one call should be sufficient. In our
experiments with TPC-H queries, the join rule implemented that way is
typically called 6-9 times more often than expected.

*3) Transformations (i.e. logical optimization) are decoupled from
implementation (i.e. physical optimization)*
Normally, you would like to mix both logical and physical rules in a single
optimization program, because it is required for proper planning. That is,
you should consider both (Ax(BxC)) and ((AxB

Re: Volcano's problem with trait propagation: current state and future

2019-12-05 Thread Vladimir Ozerov
AbstractConverter-s are attractive because they effectively emulate
straightforward recursive top-down optimization (Volcano/Cascades). But
instead of doing it with a recursive method call, which preserves the
context, we do this in Calcite as a sequence of unrelated rule calls, thus
losing the context. So with my current understanding, it could be thought
of not as a search space explosion, but rather than the inefficient
implementation of an otherwise straightforward parent->child->parent
navigation, since we achieve this navigation indirectly through the rule
queue, rather than through a normal method call. In any case, the net
result is wasted CPU. Perhaps this is not exponential waste, but some
multiplication of otherwise optimal planning. As I mentioned, in our
experiments with TPC-H, we observed a constant factor between 6-9x between
the number of joins and the number of join implementation rule invocations.
It doesn't growth past 9 even for complex queries, so I hope that this is
not an exponent :-)

Speaking of logical vs physical optimization, IMO it makes sense in some
cases. E.g. when doing predicate pushdown, you do not want to consider
intermediate logical tree states for implementation, until the predicate
reaches its final position. So running separate logical planning phase with
Volcano optimizer makes total sense to me, because it effectively prunes a
lot of not optimal logical plans before they reach the physical planning
stage. The real problem to me is that we forced to remove join planning
from the physical optimization stage. Because the goal of join planning not
to generate a single optimal plan, like with predicate pushdown, but rather
to generate a set of logical plans all of which should be implemented and
estimated. And with AbstractConverter-s this is not possible because of
their multiplicator increases the rate of search space growth, making join
planning inapplicable even for the small number of relations. So we have to
move them to the logical planning stage and pick only one permutation for
physical planning.


чт, 5 дек. 2019 г. в 15:35, Roman Kondakov :

> Vladimir,
>
> thank you for bringing it up. We are facing the same problems in Apache
> Ignite project
> and it would be great if Apache Calcite community will propose a
> solution for this
> issue.
>
>  From my point of view an approach with abstract converters looks more
> promising, but as
> you mentioned it suffers from polluting the search space. The latter can
> be mitigated by
> splitting a planning stage into the several phases: you shouldn't
> register all logical rules in the planner simultaneously - it looks like
> it is better to have several iterations of planning stage with different
> sets of rules, as Drill does.
>
> Also I'd like to mention that decoupling the logical planning from the
> physical one looks
> a bit weird to me because it violates the idea of Cascades framework.
> Possibly this decoupling is the consequence of some performance issues.
>
>
> --
> Kind Regards
> Roman Kondakov
>
> On 05.12.2019 14:24, Vladimir Ozerov wrote:
> > Hi,
> >
> > As I mentioned before, we are building a distributed SQL engine that uses
> > Apache Calcite for query optimization. The key problem we faced is the
> > inability to pull the physical traits of child relations efficiently. I'd
> > like to outline my understanding of the problem (I guess it was already
> > discussed multiple times) and ask the community to prove or disprove the
> > existence of that problem and its severity for the products which uses
> > Apache Calcite and ask for ideas on how it could be improved in the
> future.
> >
> > I'll start with the simplified problem description and mentioned more
> > complex use cases then. Consider that we have a logical tree and a set of
> > implementation rules. Our goal is to find the optimal physical tree by
> > applying these rules. The classical Cascades-based approach directs the
> > optimization process from the top to the bottom (hence "top-down").
> > However, the actual implementation of tree nodes still happens bottom-up.
> > For the tree L1 <- L2, we enter "optimize(L1)", which recursively
> delegates
> > to "optimize(L2)". We then implement children nodes L1 <- [P2', P2''],
> and
> > return back to the parent, which is now able to pick promising
> > implementations of the children nodes and reject bad ones with the
> > branch-and-bound approach. AFAIK Pivotal's Orca works this way.
> >
> > The Apache Calcite is very different because it doesn't allow the
> recursion
> > so that we lose the context on which node requested the child
> > transformation. This loss 

Re: Re: Re: Volcano's problem with trait propagation: current state and future

2019-12-06 Thread Vladimir Ozerov
cal implementations of child nodes recursively
List leftPhysicalNodes =
leftLogicalNode.optimizePhysical(maxChildCost);
List rightPhysicalNodes =
rightLigicalNode.optimizePhysical(maxChildCost);

for (RelNode leftPhysicalNode : leftPhysicalNodes) {
for (RelNode rightPhysicalNode : rightPhysicalNodes) {
// *Pruning #2*: Combination of physical input costs is already
too expensive, give up
Cost physicalLeftCost = optimizer.getCost(leftPhysicalNode);
Cost physicalRightCost = optimizer.getCost(rightPhysicalNode);

if (logicalILeftCost + logicalRightCost > maxChildCost) {
continue.
}

// Implement possible physical nodes for the given pair of
inputs (maybe more than one)
List physicalJoins = implement(leftPhysicalNode,
rightPhysicalNode);

for (RelOptRule physicalJoin : physicalJoins) {
   // *Pruning #3*: Do not consider implementation if we have
another one with the same trait set and smaller cost)
Cost physicalJoinCost = optimizer.getCost(physicalJoin);
Cost bestCostForTraitSet =
equivalenceSet.getBestCost(physicalJoin.getTraitSet());

if (physicalJoinCost > bestCostForTraitSet) {
continue.
}

// This is a good implementation. Register it in the set,
updating per-traitset best costs.
equivalenceSet.register(physicalJoin);
}
}
}

// Return the best registered expressions with different traitsets from
the current set.
return equivalenceSet.getBestExps();
}

This is a very rough pseudo-code, only to demonstrate the basic idea on how
proper bottom-up propagation not only helps us set proper traits for the
new physical node but also ensures that not optimal plans are pruned as
early as possible. Real implementation should be better abstracted and
accept enforcers as well.

Also, please notice that the pseudo-code doesn't show when logical rules
are fired. This is a separate question. One possible straightforward way is
to add the aforementioned physical routine to normal Volcano flow:
1) Fire logical rule on a node and register new nodes
2) Fire physical optimization as shown above, then invoke
"call.transformTo()" for every returned physical rel
3) Re-trigger the process for newly created nodes and their parents

A better approach is to interleave logical and physical optimizations, so
they trigger each other recursively. But this would require a serious
redesign of a "rule queue" concept.

Does it have any common points with your proposal?

Regards,
Vladimir.

[1]
https://ponymail-vm.apache.org/_GUI_/thread.html/79dac47ea50b5dfbd3f234e368ed61d247fb0eb989f87fe01aedaf25@%3Cdev.calcite.apache.org%3E


пт, 6 дек. 2019 г. в 05:41, Haisheng Yuan :

> Oh, I forgot to mention that the join planning/reordering is not a big
> problem. Calcite just use the rule to generate a single alternative plan,
> which is not ideal.  But we can't say Calcite is doing wrong.
>
> Ideally, we expect it generates multiple (neither all, nor single)
> bipartie graphs. The join reordering rule will cut each part into bipartie
> recursively and apply JoinCommutativity rule to generate more alternatives
> for each RelSet. It is just a different strategy. We can modify the rule,
> or create new join reordering rule to generate multiple plan alternatives.
>
> - Haisheng
>
> --
> 发件人:Haisheng Yuan
> 日 期:2019年12月06日 09:07:43
> 收件人:Vladimir Ozerov; dev@calcite.apache.org (
> dev@calcite.apache.org)
> 主 题:Re: Re: Volcano's problem with trait propagation: current state and
> future
>
> Generally agree with what Vladimir said. I think what Calcite has is
> logical optimization or exploration, there are almost no physical
> optimization, Calcite leaves it to third party implementators. One of my
> friends at University of Wisconsin–Madison database research group told me
> that they gave up the idea of using Calcite in their project due to this
> reason.
>
> Currently physical properties are requested in implementation rules, or
> even logical exploration rules, But each rule is independent, the
> pattern-matched expression is not aware of what does the parent operator
> want. Using AbstractConverter doesn't help, and is not promising.
>
> >> You shouldn't regiester all logical rules in the planner
> simultaneously,... as Drill does.
> That is because Calcite does too many redundant or duplicate rule
> matching, like all kinds of transpose (can't be avoided due to current
> design), matching physical operators.
>
> >> decoupling the logical planning from the physical one looks
> a bit weird to me because it violates t

Re: Re: Re: Volcano's problem with trait propagation: current state and future

2019-12-06 Thread Vladimir Ozerov
"all we know is their *collations*" -> "all we know is their *traits*"

пт, 6 дек. 2019 г. в 12:57, Vladimir Ozerov :

> Hi Haisheng,
>
> Thank you for your response. Let me elaborate my note on join planning
> first - what I was trying to say is not that rules on their own have some
> deficiencies. What I meant is that with current planner implementation,
> users tend to separate join planning from the core optimization process
> like this in the pseudo-code below. As a result, only one join permutation
> is considered during physical planning, even though join rule may
> potentially generate multiple plans worth exploring:
>
> RelNode optimizedLogicalNode = doJoinPlanning(logicalNode);
> RelNode physicalNode = doPhysicalPlanning(optimizedLogicalNode);
>
> Now back to the main question. I re-read your thread about on-demand trait
> propagation [1] carefully. I'd like to admit that when I was reading it for
> the first time about a month ago, I failed to understand some details due
> to poor knowledge of different optimizer architectures. Now I understand it
> much better, and we definitely concerned with exactly the same problem. I
> feel that trait pull-up might be a step in the right direction, however, it
> seems to me that it is not the complete solution. Let me try to explain why
> I think so.
>
> The efficient optimizer should try to save CPU as much as possible because
> it allows us to explore more plans in a sensible amount of time. To achieve
> that we should avoid redundant operations, and detect and prune inefficient
> paths aggressively. As far as I understand the idea of trait pull-up, we
> essentially explore the space of possible physical properties of children
> nodes without forcing their implementation. But after that, the Calcite
> will explore that nodes again, now in order to execute implementation
> rules. I.e. we will do two dives - one to enumerate the nodes (trait
> pull-up API), and the other one to implement them (implementation rules),
> while in Cascades one dive should be sufficient since exploration invokes
> the implementation rules as it goes. This is the first issue I see.
>
> The second one is more important - how to prune inefficient plans?
> Currently, nodes are implemented independently and lack of context doesn't
> allow us to estimates children's costs when implementing the parent, hence
> branch-and-bound is not possible. Can trait pull-up API "List
> deriveTraitSets(RelNode, RelMetadataQuery)" help us with this? If the
> children nodes are not implemented before the pull-up, all we know is their
> collations, but not their costs. And without costs, pruning is not
> possible. Please let me know if I missed something from the proposal.
>
> The possible architecture I had in mind after reading multiple papers,
> which may answer all our questions, could look like this:
> 1) We have a queue of nodes requiring optimization. Not a queue of rules.
> initial queue state is formed from the initial tree, top-down.
> 2) The node is popped from the queue, and we enter
> "node.optimize(maxCost)" call. It checks for matching rules, prioritizes
> them, and start their execution on by one. Execution of rules may re-insert
> the current node into the queue, in which case this step is repeated,
> possibly many times
> 3) Logical-logical rules (transformations) produce new logical nodes and
> put them into the queue for further optimization
> 4) Logical-physical rules (implementation) do the following:
> 4.1) Costs of logical children are estimated. The cost of a logical node
> should be less than any cost of a possible physical node that may be
> produced out of it. If the logical cost exceeds "maxCost", we stop and
> return. The whole logical subspace is pruned even before exploration.
> 4.2) Recursively call "childNode.optimize(maxCost - currentLogicalCost)"
> method, which returns a set of possible physical implementations of a
> child. Returned physical children are already registered in proper
> set/subset, but are not used for any pattern-matching, and doesn't trigger
> more rule calls!
> 4.3) Implementation rule checks the cost of the physical child. If it is
> greater than any other already observed child with the same traits, or
> exceeds the "maxCost", it is discarded. Otherwise, the physical
> implementation of the current node is produced and registered in the
> optimizer.
>
> The pseudocode for physical implementation flow for join (two inputs):
>
> Collection optimizePhysical(Cost maxCost) {
> // Estimated minimal self-cost. Any physical implementation of this
> node should have greater self-cost
> Cost logicalSelfCost = optimizer.get

Re: Re: Re: Volcano's problem with trait propagation: current state and future

2019-12-08 Thread Vladimir Ozerov
Hi Stamatis,

Thank you for the idea about Hive. I looked at it some time ago and the
codebase was substantially more complex to understand for me than in other
projects, so I gave up. I'll try to do the analysis again.
I'd like to mention that I also had a thought that maybe the implementation
of a top-down optimization is not a concern of VolcanoPlanner, and the
brand new planner may play well here. But from a practical perspective, of
course, I keep a hope that we will find a less intrusive way to
introduce efficient physical optimization into VolcanoPlanner :-)

Regards,
Vladimir.

вс, 8 дек. 2019 г. в 12:42, Stamatis Zampetakis :

> Thanks Vladimir for this great summary. It is really helpful to know how
> the different projects use the optimizer and it certainly helps to identify
> limitations on our implementation.
>
> I cannot provide any valuable feedback at the moment since I have to find
> some time to read more carefully your analysis.
>
> In the meantime, I know that Hive is also using Calcite for quite some time
> now so maybe you can get some new ideas (or complete your background study)
> by looking in their code.
>
> @Haisheng: I think many people did appreciate the discussion for pull up
> traits so I wouldn't say that we abandoned it. I had the impression that we
> were waiting a design doc.
>
> In general it may not be feasible to cover all use cases with a single
> optimizer. I wouldn't find it bad to introduce another planner if there are
> enough reasons to do so.
>
> Best,
> Stamatis
>
>
> On Fri, Dec 6, 2019, 11:00 AM Vladimir Ozerov  wrote:
>
> > "all we know is their *collations*" -> "all we know is their *traits*"
> >
> > пт, 6 дек. 2019 г. в 12:57, Vladimir Ozerov :
> >
> > > Hi Haisheng,
> > >
> > > Thank you for your response. Let me elaborate my note on join planning
> > > first - what I was trying to say is not that rules on their own have
> some
> > > deficiencies. What I meant is that with current planner implementation,
> > > users tend to separate join planning from the core optimization process
> > > like this in the pseudo-code below. As a result, only one join
> > permutation
> > > is considered during physical planning, even though join rule may
> > > potentially generate multiple plans worth exploring:
> > >
> > > RelNode optimizedLogicalNode = doJoinPlanning(logicalNode);
> > > RelNode physicalNode = doPhysicalPlanning(optimizedLogicalNode);
> > >
> > > Now back to the main question. I re-read your thread about on-demand
> > trait
> > > propagation [1] carefully. I'd like to admit that when I was reading it
> > for
> > > the first time about a month ago, I failed to understand some details
> due
> > > to poor knowledge of different optimizer architectures. Now I
> understand
> > it
> > > much better, and we definitely concerned with exactly the same
> problem. I
> > > feel that trait pull-up might be a step in the right direction,
> however,
> > it
> > > seems to me that it is not the complete solution. Let me try to explain
> > why
> > > I think so.
> > >
> > > The efficient optimizer should try to save CPU as much as possible
> > because
> > > it allows us to explore more plans in a sensible amount of time. To
> > achieve
> > > that we should avoid redundant operations, and detect and prune
> > inefficient
> > > paths aggressively. As far as I understand the idea of trait pull-up,
> we
> > > essentially explore the space of possible physical properties of
> children
> > > nodes without forcing their implementation. But after that, the Calcite
> > > will explore that nodes again, now in order to execute implementation
> > > rules. I.e. we will do two dives - one to enumerate the nodes (trait
> > > pull-up API), and the other one to implement them (implementation
> rules),
> > > while in Cascades one dive should be sufficient since exploration
> invokes
> > > the implementation rules as it goes. This is the first issue I see.
> > >
> > > The second one is more important - how to prune inefficient plans?
> > > Currently, nodes are implemented independently and lack of context
> > doesn't
> > > allow us to estimates children's costs when implementing the parent,
> > hence
> > > branch-and-bound is not possible. Can trait pull-up API
> > "List
> > > deriveTraitSets(RelNode, RelMetadataQuery)" help us with this? If the
> > > children nodes are not 

Re: Re: Re: Volcano's problem with trait propagation: current state and future

2019-12-09 Thread Vladimir Ozerov
After looking at Hive implementation I have the impression that it doesn't
use Apache Calcite for physical planning, hence it doesn't have the
problems mentioned in this topic.

вс, 8 дек. 2019 г. в 18:55, Vladimir Ozerov :

> Hi Stamatis,
>
> Thank you for the idea about Hive. I looked at it some time ago and the
> codebase was substantially more complex to understand for me than in other
> projects, so I gave up. I'll try to do the analysis again.
> I'd like to mention that I also had a thought that maybe the
> implementation of a top-down optimization is not a concern of
> VolcanoPlanner, and the brand new planner may play well here. But from a
> practical perspective, of course, I keep a hope that we will find a less
> intrusive way to introduce efficient physical optimization into
> VolcanoPlanner :-)
>
> Regards,
> Vladimir.
>
> вс, 8 дек. 2019 г. в 12:42, Stamatis Zampetakis :
>
>> Thanks Vladimir for this great summary. It is really helpful to know how
>> the different projects use the optimizer and it certainly helps to
>> identify
>> limitations on our implementation.
>>
>> I cannot provide any valuable feedback at the moment since I have to find
>> some time to read more carefully your analysis.
>>
>> In the meantime, I know that Hive is also using Calcite for quite some
>> time
>> now so maybe you can get some new ideas (or complete your background
>> study)
>> by looking in their code.
>>
>> @Haisheng: I think many people did appreciate the discussion for pull up
>> traits so I wouldn't say that we abandoned it. I had the impression that
>> we
>> were waiting a design doc.
>>
>> In general it may not be feasible to cover all use cases with a single
>> optimizer. I wouldn't find it bad to introduce another planner if there
>> are
>> enough reasons to do so.
>>
>> Best,
>> Stamatis
>>
>>
>> On Fri, Dec 6, 2019, 11:00 AM Vladimir Ozerov  wrote:
>>
>> > "all we know is their *collations*" -> "all we know is their *traits*"
>> >
>> > пт, 6 дек. 2019 г. в 12:57, Vladimir Ozerov :
>> >
>> > > Hi Haisheng,
>> > >
>> > > Thank you for your response. Let me elaborate my note on join planning
>> > > first - what I was trying to say is not that rules on their own have
>> some
>> > > deficiencies. What I meant is that with current planner
>> implementation,
>> > > users tend to separate join planning from the core optimization
>> process
>> > > like this in the pseudo-code below. As a result, only one join
>> > permutation
>> > > is considered during physical planning, even though join rule may
>> > > potentially generate multiple plans worth exploring:
>> > >
>> > > RelNode optimizedLogicalNode = doJoinPlanning(logicalNode);
>> > > RelNode physicalNode = doPhysicalPlanning(optimizedLogicalNode);
>> > >
>> > > Now back to the main question. I re-read your thread about on-demand
>> > trait
>> > > propagation [1] carefully. I'd like to admit that when I was reading
>> it
>> > for
>> > > the first time about a month ago, I failed to understand some details
>> due
>> > > to poor knowledge of different optimizer architectures. Now I
>> understand
>> > it
>> > > much better, and we definitely concerned with exactly the same
>> problem. I
>> > > feel that trait pull-up might be a step in the right direction,
>> however,
>> > it
>> > > seems to me that it is not the complete solution. Let me try to
>> explain
>> > why
>> > > I think so.
>> > >
>> > > The efficient optimizer should try to save CPU as much as possible
>> > because
>> > > it allows us to explore more plans in a sensible amount of time. To
>> > achieve
>> > > that we should avoid redundant operations, and detect and prune
>> > inefficient
>> > > paths aggressively. As far as I understand the idea of trait pull-up,
>> we
>> > > essentially explore the space of possible physical properties of
>> children
>> > > nodes without forcing their implementation. But after that, the
>> Calcite
>> > > will explore that nodes again, now in order to execute implementation
>> > > rules. I.e. we will do two dives - one to enumerate the nodes (trait
>> > > pull-up API), and the other one to implement them (implementation
>> r

Re: Re: Re: Volcano's problem with trait propagation: current state and future

2019-12-10 Thread Vladimir Ozerov
HI Stamatis,

This paper doesn't explain the Calcite integration in detail required for
the purpose of our discussion. From what I see in the code, VolcanoPlanner
is only used for view materialization planning, which is also mentioned in
the paper. All other optimizations stages, including join rewrite, use HEP
approach. In other words, there is no mixed logical-physical planning in a
single optimization run.

This is my current understanding of the Hive's codebase. It would be nice
if Hive maintainers would confirm or disprove that.

Regards,
Vladimir.

вт, 10 дек. 2019 г. в 02:22, Stamatis Zampetakis :

> It's been some time that I didn't look into the code but the most recent
> Hive paper [1] mostly talks about Calcite in the query optimization section
> so I have to say I am a bit surprised.
>
> [1] https://arxiv.org/pdf/1903.10970.pdf
>
> On Mon, Dec 9, 2019 at 6:21 PM Vladimir Ozerov  wrote:
>
> > After looking at Hive implementation I have the impression that it
> doesn't
> > use Apache Calcite for physical planning, hence it doesn't have the
> > problems mentioned in this topic.
> >
> > вс, 8 дек. 2019 г. в 18:55, Vladimir Ozerov :
> >
> > > Hi Stamatis,
> > >
> > > Thank you for the idea about Hive. I looked at it some time ago and the
> > > codebase was substantially more complex to understand for me than in
> > other
> > > projects, so I gave up. I'll try to do the analysis again.
> > > I'd like to mention that I also had a thought that maybe the
> > > implementation of a top-down optimization is not a concern of
> > > VolcanoPlanner, and the brand new planner may play well here. But from
> a
> > > practical perspective, of course, I keep a hope that we will find a
> less
> > > intrusive way to introduce efficient physical optimization into
> > > VolcanoPlanner :-)
> > >
> > > Regards,
> > > Vladimir.
> > >
> > > вс, 8 дек. 2019 г. в 12:42, Stamatis Zampetakis :
> > >
> > >> Thanks Vladimir for this great summary. It is really helpful to know
> how
> > >> the different projects use the optimizer and it certainly helps to
> > >> identify
> > >> limitations on our implementation.
> > >>
> > >> I cannot provide any valuable feedback at the moment since I have to
> > find
> > >> some time to read more carefully your analysis.
> > >>
> > >> In the meantime, I know that Hive is also using Calcite for quite some
> > >> time
> > >> now so maybe you can get some new ideas (or complete your background
> > >> study)
> > >> by looking in their code.
> > >>
> > >> @Haisheng: I think many people did appreciate the discussion for pull
> up
> > >> traits so I wouldn't say that we abandoned it. I had the impression
> that
> > >> we
> > >> were waiting a design doc.
> > >>
> > >> In general it may not be feasible to cover all use cases with a single
> > >> optimizer. I wouldn't find it bad to introduce another planner if
> there
> > >> are
> > >> enough reasons to do so.
> > >>
> > >> Best,
> > >> Stamatis
> > >>
> > >>
> > >> On Fri, Dec 6, 2019, 11:00 AM Vladimir Ozerov 
> > wrote:
> > >>
> > >> > "all we know is their *collations*" -> "all we know is their
> *traits*"
> > >> >
> > >> > пт, 6 дек. 2019 г. в 12:57, Vladimir Ozerov :
> > >> >
> > >> > > Hi Haisheng,
> > >> > >
> > >> > > Thank you for your response. Let me elaborate my note on join
> > planning
> > >> > > first - what I was trying to say is not that rules on their own
> have
> > >> some
> > >> > > deficiencies. What I meant is that with current planner
> > >> implementation,
> > >> > > users tend to separate join planning from the core optimization
> > >> process
> > >> > > like this in the pseudo-code below. As a result, only one join
> > >> > permutation
> > >> > > is considered during physical planning, even though join rule may
> > >> > > potentially generate multiple plans worth exploring:
> > >> > >
> > >> > > RelNode optimizedLogicalNode = doJoinPlanning(logicalNode);
> > >> > > RelNode physicalNode = doPhysicalPlanning(optimizedL

Re: Re: Re: Re: Volcano's problem with trait propagation: current state and future

2019-12-12 Thread Vladimir Ozerov
et(join.getInput(0), leftReq,
maxCost - logicalCost);

for (RelNode leftNode : leftNodes) {
// Recursive call to the function from p.2, exploring the right
child. The cost is adjusted even more aggressively.
List rightNodes = optimizeSet(join.getInput(1),
rightReq, maxCost - logicalCost - leftNode.getCost());

for (RelNode rightNode : rightNodes) {
// Create physical rel.
RelNode physicalJoin = implementHashJoin(logicalJoin,
leftNode, rightNode);

// One more attempt to prune.
if (physicalJoin.getCost() <= maxCost) {
// Ok, this is a good physical plan. Register in the
equivalence set.
logicalJoin.getEquivalenceSet().register(physicalJoin);
}
}
}
}
}

I understand this pseudo-code might be hard to follow, but unfortunately, I
do not have any working prototype at the moment. The key takeaways:
1) The optimization process is guided by parents, the context is always
preserved
2) Aggressive caching of already executed requests. Note that this is not
MEMO, but the additional cache to prevent excessive rule executions
3) Aggressive pruning - the cost is propagated top-down
4) Finally, it closely follows your idea of on-demand traits - we really
demand traits from children nodes.  But this proposal also propagates costs
to allow for branch-and-bound, and also optimize children nodes while
pulling up their traits, thus saving optimization time.

I would appreciate the community feedback since I still feel that I miss
some important details.

Regards,
Vladimir.

вт, 10 дек. 2019 г. в 04:45, Haisheng Yuan :

> Hi Vladimir,
>
> Sorry for my late reply.
> WRT join planning, it is not required to put join reordering rule into the
> HEP planner. It can also be put into Volcano planner. Indeed, it is not
> ideal for the join ordering rule to generate a single plan. We can create a
> nother rule to generate multiple alternatives and put the rule into Volcano
> planner. This way you can get what you want.
>
> The pull-up trait is not the essence of on-demand trait request, the main
> idea is link [1].
>
> >> 4.1) If the logical cost exceeds "maxCost", we stop and return. The
> whole logical subspace is pruned even before exploration.
> In many cases, the search space you pruned is just the specific operator,
> because the child operator should be a MEMO group, other parent operators
> might still need to explore it, especially when the JoinReorderingRule only
> generate a single logical optimal join order.
>
> >> 4.2) Returned physical children are already registered in proper
> set/subset, but are not used for any pattern-matching, and doesn't trigger
> more rule calls!
> That is the problem of Calcite's default behaviour. Most of the rules'
> default INSTANCE provided by Calcite not only match logical operators but
> also physical operators. I am against that. I am not sure if you have
> created your own rule instances or not.
>
> >> 4.3) Implementation rule checks the cost of the physical child.
> During implementation rule, it is possiple that we are not able to
> calculate the cost yet. Depending on the rule match order, if it is
> top-down rule matching, the child operators are still logical. If it is
> bottom-up rule matching, the child operators are still not enforced, say we
> generate a MergeJoin with 2 children not sorted yet, how do we estimate the
> cost?
>
> >> If it is greater than any other already observed child with the same
> traits
> How can we observe it inside the implementation rule?
>
> [1]
> http://mail-archives.apache.org/mod_mbox/calcite-dev/201910.mbox/%3cd75b20f4-542a-4a73-897e-66ab426494c1.h.y...@alibaba-inc.com%3e
>
> - Haisheng
>
> --
> 发件人:Vladimir Ozerov
> 日 期:2019年12月06日 18:00:01
> 收件人:Haisheng Yuan
> 抄 送:dev@calcite.apache.org (dev@calcite.apache.org) >
> 主 题:Re: Re: Re: Volcano's problem with trait propagation: current state
> and future
>
> "all we know is their *collations*" -> "all we know is their *traits*"
>
> пт, 6 дек. 2019 г. в 12:57, Vladimir Ozerov :
>
>> Hi Haisheng,
>>
>> Thank you for your response. Let me elaborate my note on join planning
>> first - what I was trying to say is not that rules on their own have some
>> deficiencies. What I meant is that with current planner implementation,
>> users tend to separate join planning from the core optimization process
>> like this in the pseudo-code below. As a result, only one join permutation
>> is considered during physical planning, even though join rule may
>> potentially generate multiple

Re: Re: Re: [DISCUSS] Proposal to add API to force rules matching specific rels

2020-03-14 Thread Vladimir Ozerov
Hi Roman,

In my understanding, the proposed minor changes may only decrease the total
number of rule invocations slightly, but all principal problems remain the
same. In the top-down approach, you do not want to implement bottom logical
nodes unless it is requested explicitly by a parent operator.

It seems to me that the very first step to efficient optimizer could be a
new rule invocation infrastructure. There should be *no global rule queue*
at all. Instead, we may introduce the per-node rule queue. Then, the
optimizer performs a recursive top-down optimization dive, invoking the
rules for every operator. Consider the following simple tree:
-- LogicalProject
 LogicalScan

Assuming that we have two implementation rules ProjectRule, ScanRule, and
abstract converters enabled, VolcanoOptimizer will proceed as follows,
generating one unnecessary rule call:
1. Define global rule call queue: ProjectRule, ScanRule
2. Call ProjectRule, no new nodes are produced
3. Call ScanRule, produce physical scans, reschedule ProjectRule
4. Call ProjectRule again, produce the physical project

With the proposed approach, it will work differently:
1. Define per-operator queues:
LogicalProject -> ProjectRule
LogicalScan -> ScanRule
2. Call optimize(LogicalProject)
3. Invoke ProjectRule, which calls optimize(LogicalScan) on the input
4. Invoke ScanRule, produce physical scans, return control back to
ProjectRule
5. Produce the physical project, finish optimization

Now we have only 2 rule invocations as expected, and we reached the same
result as in the proposed minor changes. But the crucial difference is that
now we have well-defined control flow between operators: start at the top,
delegate to children. With this infrastructure in place, we will be able to
introduce more complex features, such as pruning, or partial exploration
later on.

But notice that this change will be incompatible with the current rules,
i.e. they should be re-written for the new optimization algorithm (e.g. see
step 3), which might be a blocker for the current Calcite users. So maybe
it is better to think of a new optimizer, rather than fixing
VolcanoOptimizer.

Regards,
Vladimir.


вт, 14 янв. 2020 г. в 23:52, Haisheng Yuan :

> On the other hand, if we don't preprocess and normalize the rel expression
> before going to valcano planner, still compute and keep logical/relational
> properties, like cardinality, for each operator, how can we achieve group
> seach space pruning? Given a physical group expression, its required
> property and upper bound cost C_limit, we need to get its child group
> cardinality, compute local cost C_local,  so that we can calculate the
> child group's upper bound cost and pass it down.
>
> I can't figure out how we can do group pruning without shared relational
> properties.
>
> - Haisheng
>
> --
> 发件人:Haisheng Yuan
> 日 期:2020年01月14日 12:06:17
> 收件人:dev@calcite.apache.org
> 主 题:Re: Re: [DISCUSS] Proposal to add API to force rules matching specific
> rels
>
> The example is valid if Calcite doesn't do normalization or preprocessing
> before going to VolcanoPlanner.
> But many databases and big data systems will try to preprocess the
> expression (push down predicates etc.) so that expressions in the same
> group can share the logical properties, for most case if not all. You may
> argue that it should be cost based, e.g. evaluating filter early can still
> be bad. It is true, but how accurate will the statistics be, how accurate
> will the cost model be?
>
> - Haisheng
>
> --
> 发件人:Julian Hyde
> 日 期:2020年01月13日 08:44:54
> 收件人:dev@calcite.apache.org
> 主 题:Re: [DISCUSS] Proposal to add API to force rules matching specific rels
>
> > MEMO group (RelSet) represents logically equivalent expressions.
> > All the expressions in one group should share the same logical
> > properties, e.g. functional dependency, constraint properties etc.
> > But they are not sharing it. Computation is done for each individual
> operator.
>
> It's good, and correct, that we compute for each individual operator.
>
> Suppose that a RelSubset s contains RelNode r1 and we know that the
> constraint x > 0 holds. Suppose that we also have r2 with constraint y
> < 10, and we discover that r1 and r2 are equivalent and belong
> together in s. Now we can safely say that both constraints (x > 0 and
> y < 10) apply to both r1 and r2.
>
> Deducing additional constraints in this way is a big win. The effort
> to compute constraints for each RelNode is well-spent.
>
> This kind of deduction applies to other logical properties (e.g.
> unique keys) and it applies to RelSet as well as RelSubset.
>
> Julian
>
>
> On Sun, Jan 12, 2020 at 10:10 AM Roman Kondakov
>  wrote:
> >
> > @Haisheng
> >
> > > Calcite uses Project operator and all kinds of ProjectXXXTranposeRule
> to prune unused columns.
> >
> > I also noticed that in most cases Proj

Re: Re: Re: Re: [DISCUSS] Proposal to add API to force rules matching specific rels

2020-03-14 Thread Vladimir Ozerov
Hi Haisheng,

You are right, the behavior I showed is not the default one, I should
provide more context here. This is how we (Hazelcast) and at least Drill
use the engine to ensure that the produced plan is optimal, I gave an
example in [1].
In real distributed engines, we rely on physical properties heavily. Most
notably, distribution and collation. And the fundamental problem with the
VolcanoOptimizer, is that it cannot propagate traits in a controlled
manner. This, in turn, forces us to use AbstractConverters and implement
rules in ways, which appear strange to Calcite community :-). And this, in
turn, leads to excessive rule calls and failure to plan complex queries.

Let's consider the same tree again, but now assuming that this is not the
complete tree, but a subtree, and there are some parent operators. Let's
also assume that the ScanRule may produce two equivalent operators with
different physical properties: PhysicalScan and PhysicalIndexScan[a ASC].
It is important to consider both alternatives in parent operators. Now
let's consider two different ways to optimize that subtree.

1. Canonical Calcite way (default)
1.1 Perform initial rules ordering, parent rules fire first: [ProjectRule,
ScanRule]
1.2 Invoke ProjectRule, which produces physical project without any
physical traits
1.3 Invoke ScanRule, which produces, PhysicalScan and PhysicalIndexScan[a
ASC]
Since ProjectRule was not aware of different scan alternatives, it missed
collation, and resulting hypergraph looks like this:

-- PhysicalProject
 [PhysicalScan, PhysicalIndexScan[a ASC]]

This plan is suboptimal, because of parent operators cannot take advantage
of collation.

2. Hazelast/Drill way:
2.1 Enable abstract converters
2.2 Rules are ordered in the same way as in example 1: [ProjectRule,
ScanRule]
2.3 ProjectRule fires, enumerates physical implementations of the input.
Since there are no physical inputs yet, it exits without any transformations
2.4 ScanRule fires and produces two physical scans
2.5 Abstract converters ensure that the ProjectRule is scheduled for
execution again because it's input RelSet has new nodes
2.6 ProjectRule fires again, now having physical inputs, and generates one
implementation per unique combination of physical input traits.

As a result, we have two graphs now:

Graph 1:
-- PhysicalProject
 PhysicalScan

Graph 2:
-- PhysicalProject[a ASC]
 PhysicalIndexScan[a ASC]

Notice how we propagated physical collation. Now parent operators may take
advantage of it, e.g. eliminate sorting, or perform streaming aggregation
instead of blocking hash aggregation.

This is the fundamental problem we have in Hazelcast: how to ensure the
complete exploration of the search space without excessive rule calls.

Very short summary:
1. The default behavior of VolcanoOptimizer cannot explore the physical
search space, so plans are not optimal
2. Abstract converters fix this if you follow a certain pattern in rule
implementations (see 2.3), but generate too many rule calls, so join
planning rules cannot be called together with other rules, which again lead
to not optimal plans (yet, better than with p.1)
3. "Trait pull-up" proposal may fix it. But I have a feeling that pulling
up possible trait combinations from a child node is indistinguishable from
child node exploration, so it may be not very efficient again
4. A brand new optimizer implementation with recursive top-down approach
may address all the concerns from p.1-p.3, but appears to be complex to
implement and may be incompatible with existing rules

Not an easy choice.

Regards,
Vladimir.

[1]
https://mail-archives.apache.org/mod_mbox/calcite-dev/201910.mbox/%3CCAJJmzpU9_75O48WeEKgHKg3fTMhK3eAMmHNOVvczj_uUTBxHkA%40mail.gmail.com%3E

сб, 14 мар. 2020 г. в 21:53, Haisheng Yuan :

> I agree that there should be no global rule queue, we should it do it on
> per-operator basis, which is also how other major Cascades frameworks do.
>
> However, Calcite's VolcanoPlanner doesn't generate unnecessary rule calls
> as you described. The current process is:
> 1. global rule queue: ScanRule, ProjectRule
> 2. Call ScanRule, produce physical scan
> 3. Call ProjectRule, produce physical project.
>
> Even with global rule queue of reversed order ProjectRule, ScanRule, there
> are still 2 rule calls. In your step 2, ProjectRule doesn't produce
> physical node, which is incorrect. Any rule is, and should be independent
> with each other rule. If your rule relies on other operators or rules to be
> explored first, then you should think about it twice.
>
> - Haisheng
>
> --
> 发件人:Vladimir Ozerov
> 日 期:2020年03月15日 01:50:10
> 收件人:dev@calcite.apache.org (dev@calcite.apache.org) >
> 主 题:Re: Re: Re: [DISCUSS] Proposal to add API to force rules matching
> specific rels
>
> Hi Roman,
>
> In 

Re: Set of k-cheapest plans

2020-03-19 Thread Vladimir Ozerov
Hi Liya,

This approach may work. However, as Julian mentioned earlier, the main
question is why do you need these plans? If you collect the plans in the
way you described, you may get several best plans which are only marginally
different from each other, which are of little use.

For example, there is one real scenario where several plans might be needed
- join planning. Several Calcite-based engines perform join planning as a
separate phase because otherwise planning might take too much time. And
this phase produces only one RelNode. What we would like to have here
instead, is a set of best plans for different join orders, e.g. "A join B"
and "B join A". But in the proposed algorithm, you may easily get two
flavors of "A join B", which is not what we need.

So perhaps what we need instead, is the ability to switch off certain
RelNode-s or even RelSubset-s from the plan during the best-exp calculation?

Regards,
Vladimir.


чт, 19 мар. 2020 г. в 05:03, Fan Liya :

> IMO, there is no easy way, and the algorithm should depends on definition
> of 'alternative plans'.
>
> In general, the algorithm can proceed like this:
> 1. we use the volcano algoirthm to find the best plan
> 2. we make the self cost of some node to be infinite, and then apply the
> volcano algorithm again to find the second best plan
> 3. we repeat the above steps, until we find the k best plans.
>
> Best,
> Liya Fan
>
> On Thu, Mar 19, 2020 at 3:20 AM Julian Hyde  wrote:
>
> > There’s no easy way. You could modify ‘buildCheapestPlan()’ to build all
> > plans below a cost limit. (You’d have to carefully choose a cost limit
> only
> > a little above the optimal cost, otherwise you’ll get huge numbers of
> > plans.)
> >
> > I fear that you’ll get plans that are different in only trivial or minor
> > respects (e.g. ordering of items in a Project) whereas you probably want
> > plans with significant differences (e.g. different join orders).
> >
> > Julian
> >
> > > On Mar 18, 2020, at 12:01 PM, Rui Souto  wrote:
> > >
> > > Hi there!
> > >
> > > Just recently started to learn about Apache Calcite. What's the best
> way
> > to
> > > get a list of the k-cheapest alternative plans generated by the
> optimizer
> > > for a given query? (being k an arbitrary number)
> >
> >
>


Re: Re: Re: Re: Re: [DISCUSS] Proposal to add API to force rules matching specific rels

2020-03-19 Thread Vladimir Ozerov
Hi Haisheng,

Thank you for pointing to 1.23.0 changes, I'll try to use this version.

Regards,
Vladimir.

вс, 15 мар. 2020 г. в 08:05, Haisheng Yuan :

> Thanks for your detailed explanation, Vladimir.
>
> Indeed, the only way to propagate traits in Calcite currently is using
> rules, which is a big pain. I can feel your pain. I tried to come up ways
> to implement the trait derivation and requirement in the framwork without
> breaking current usages, only turns out it is almost impossible. It has too
> many stakeholders, even a small change may incur opposition.
>
> But before we get the real top-down cascades framework, there are still a
> lot you can do to improve your planner's performance.
>
> Since Calcite 1.22.0, I committed a change that enabes RelSubset to be
> used to trigger a rule, which can greatly reduce the number of rule calls
> for trait propagation. With your example, you need 2 rules:
> 1. Physical implementation rule
>   match pattern: operand(LogicalProject.class)
>   Produce PhysicalProject without trait
> 2. Project trait propagtion rule
>   match pattern: operand(PhysicalProject.class, operand(RelSubset.class))
>   Produce PhysicalProject with derived trait.
>
> Since 1.23.0, we removed the rule match importances and ordering, I guess
> the can reduce the optimizatino time around 10~20% for some complex queries
> with many rule calls.
>
> - Haisheng
>
> --
> 发件人:Vladimir Ozerov
> 日 期:2020年03月15日 04:18:53
> 收件人:Haisheng Yuan
> 抄 送:dev@calcite.apache.org (dev@calcite.apache.org) >
> 主 题:Re: Re: Re: Re: [DISCUSS] Proposal to add API to force rules matching
> specific rels
>
> Hi Haisheng,
>
> You are right, the behavior I showed is not the default one, I should
> provide more context here. This is how we (Hazelcast) and at least Drill
> use the engine to ensure that the produced plan is optimal, I gave an
> example in [1].
> In real distributed engines, we rely on physical properties heavily. Most
> notably, distribution and collation. And the fundamental problem with the
> VolcanoOptimizer, is that it cannot propagate traits in a controlled
> manner. This, in turn, forces us to use AbstractConverters and implement
> rules in ways, which appear strange to Calcite community :-). And this, in
> turn, leads to excessive rule calls and failure to plan complex queries.
>
> Let's consider the same tree again, but now assuming that this is not the
> complete tree, but a subtree, and there are some parent operators. Let's
> also assume that the ScanRule may produce two equivalent operators with
> different physical properties: PhysicalScan and PhysicalIndexScan[a ASC].
> It is important to consider both alternatives in parent operators. Now
> let's consider two different ways to optimize that subtree.
>
> 1. Canonical Calcite way (default)
> 1.1 Perform initial rules ordering, parent rules fire first: [ProjectRule,
> ScanRule]
> 1.2 Invoke ProjectRule, which produces physical project without any
> physical traits
> 1.3 Invoke ScanRule, which produces, PhysicalScan and PhysicalIndexScan[a
> ASC]
> Since ProjectRule was not aware of different scan alternatives, it missed
> collation, and resulting hypergraph looks like this:
>
> -- PhysicalProject
>  [PhysicalScan, PhysicalIndexScan[a ASC]]
>
> This plan is suboptimal, because of parent operators cannot take advantage
> of collation.
>
> 2. Hazelast/Drill way:
> 2.1 Enable abstract converters
> 2.2 Rules are ordered in the same way as in example 1: [ProjectRule,
> ScanRule]
> 2.3 ProjectRule fires, enumerates physical implementations of the input.
> Since there are no physical inputs yet, it exits without any
> transformations
> 2.4 ScanRule fires and produces two physical scans
> 2.5 Abstract converters ensure that the ProjectRule is scheduled for
> execution again because it's input RelSet has new nodes
> 2.6 ProjectRule fires again, now having physical inputs, and generates
> one implementation per unique combination of physical input traits.
>
> As a result, we have two graphs now:
>
> Graph 1:
> -- PhysicalProject
>  PhysicalScan
>
> Graph 2:
> -- PhysicalProject[a ASC]
>  PhysicalIndexScan[a ASC]
>
> Notice how we propagated physical collation. Now parent operators may take
> advantage of it, e.g. eliminate sorting, or perform streaming aggregation
> instead of blocking hash aggregation.
>
> This is the fundamental problem we have in Hazelcast: how to ensure the
> complete exploration of the search space without excessive rule calls.
>
> Very short summary:
> 1. The default behavior of VolcanoOptimizer cannot explore the p

Re: Using indexes rather than table scans with Calcite

2020-05-29 Thread Vladimir Ozerov
Hi,

Products that use Apache Calcite typically implement index handling on
their own, since Apache Calcite has only limited support for physical
optimization. You may implement your own index scan operator and rules that
use this operator. For example, take a look how index planning is done in
Apache Drill.

Materialised views is a hacky way to achieve this, since normally they are
used for very different purposes.

пт, 29 мая 2020 г. в 11:44, Tim Fox :

> Hi,
>
> I'm building a query engine with Calcite - really enjoying working with
> Calcite so far!
>
> When creating a plan, it seems Calcite always creates a plan where the
> sources are table scans, however in my implementation the tables can have
> indexes on them so a table scan is not always the right choice.
>
> I was wondering if there was any way of making Calcite "index aware" - e.g.
> perhaps providing hints to the table scan instance that, actually, an index
> scan or a primary key lookup should be used instead of actually scanning
> the table. E.g. On the table meta-data if we provided information about any
> indexes on the table, then Calcite could figure out what parts of the query
> to push to the table scan and which to keep in the rest of the plan.
>
> There are two specific cases I really care about:
>
> 1. Queries that contain a primary key lookup:
>
> select * from some_table where key_column=23 AND some_other_column='foo';
>
> In the above case the 'select * from some_table where key_column=23' can be
> implemented as a simple PK lookup in the source table, not requiring a
> scan, thus leaving just the filter corresponding to
> 'some_other_column='foo'' in the rest of the plan
>
> 2. Queries with expressions on a column which has a secondary index
>
> select * from users where country='UK' and some_other_column='foo';
>
> We have many users, and let's say 10% of them are from UK (still a lot). We
> have a secondary index in the country column in the source table so we can
> do an efficient index scan to retrieve the matching records.
>
> I found this document
> https://calcite.apache.org/docs/materialized_views.html which seems like
> it
> might help me in some way.
>
> The idea being if I can think of my indexes as materialized views then the
> query can be written against those materialized views as sources instead of
> the original table sources. There appears to be a rule
> 'MaterializedViewRule' that does this already (?).
>
> This seems to get me a bit further, however, for this approach to work, it
> seems I would have to create materialized views _dynamically_ during
> evaluation of the query, register them, rewrite the query, execute it, then
> deregister the materialized view.
>
> E.g. for the primary key lookup example above, for the following query:
>
> select * from some_table where key_column=23 AND some_other_column='foo';
>
> I would need to dynamically create a materialized view corresponding to:
>
> select * from some_table where key_column=23
>
> Then rewrite the query using MaterializedViewRule.
>
> In the general case, in order to figure out what materialized views I need
> to dynamically create I would need to examine the query, figure out which
> columns in expressions have indexes on them and from them work out the best
> materialized view to create based on that information. This seems non
> trivial.
>
> Does anyone have any suggestions or pointers for how to implement this kind
> of thing? I suspect I'm not the first person to have tried to do this, as
> using indexes on tables seems a pretty common thing in many systems (?)
>


Re: Using indexes rather than table scans with Calcite

2020-05-31 Thread Vladimir Ozerov
As already mentioned, the fundamental problem with "indexes as materialized
views" approach is that you have to register them beforehand, instead of
using them only when needed. On the other hand, the complexity of index
planning comes from cost estimation and predicate splitting.
Materializations cannot help you with that anyhow. This is why I call this
approach (not materialized views per se) "hacky" - you reuse several simple
parts of the Calcite infrastructure at the cost of loss in the flexibility
of the planner, while the most complicated parts still need to be
implemented by hand.

Materialized views could be a good fit e.g for partial indexes because in
this case, Calcite could help you with complex subsumption mechanics. But
for standard indexes, the pros/cons balance is not that obvious.

вс, 31 мая 2020 г. в 19:28, xu :

> Hi Tim,
>
> I am working on MySQL InnoDB adapter and trying to introduce this to
> Calcite, currently it is only in early stage, and not approved/reviewed by
> committers yet. Anyway, we are facing the same problem like what index to
> use, how to push down order by operation, etc. I have developed a simple
> rule based adapter to be "index aware" and being able to leverage a MySQL
> InnoDB storage engine written in Java. Hope this will help you to explore
> more options.
>
> https://issues.apache.org/jira/browse/CALCITE-4034
>
> Thanks,
> Xu
>
> Haisheng Yuan  于2020年5月31日周日 下午10:06写道:
>
> > Hi Roman,
> >
> > Thank you for sharing your thoughts.
> >
> > > It can be very tricky because the rule should consider not
> > > only filters, but also collations. This leads to increasing the
> > > complexity of such rules.
> >
> > Logical transformation rules like FilterTableScan2IndexTableScanRule
> > should not consider physical properties, like collation, distribution.
> You
> > forgot that we just reached consensus in CALCITE-3972. :)
> >
> > Regarding the 2nd option that uses index b, it is indeed not that easy
> for
> > Calcite 1.22.0. In latest version, it now becomes possible. After rule
> > transformation, during top-down trait request, the collation is passed
> > through Filter, down to physical TableScan, which accepts the trait
> request
> > with collation on b, find there is an index on b, and return a new
> RelNode
> > IndexScan. This process can be done in
> > EnumerableTableScan#passThrough(required).
> >
> > > I hope, when the Cascades-style optimizer become a part of Calcite, the
> > search space
> > > pollution will not be a serious issue anymore.
> >
> > I hope so too. Top-down style can help alleviate space pollution by space
> > pruning. But the space pollution caused by LogicalProject operator and
> its
> > related rules still can't be avoided. :)
> >
> > Again, thanks for sharing your experience with us.
> >
> > Haisheng
> >
> > On 2020/05/31 09:58:36, Roman Kondakov 
> > wrote:
> > > Hi Haisheng,
> > >
> > > The basic rationale behind the using materialized views for secondary
> > > index representation instead of special rules like mentioned
> > > FilterTableScan2IndexTableScanRule is the simplicity of implementation.
> > >
> > > You are absolutely right that materialized views approach has an
> obvious
> > > drawback that it should register all indexes as materialized views in
> > > the optimizer's search space. But we expect our users will not overuse
> > > indexes because in general having too many indexes is a bad practice:
> on
> > > each table update we also should update it's indexes and it can cause
> > > some performance degradation of the system as a whole, not only
> > > optimizer. So we expect the number of indexes will be relatively small.
> > >
> > > Ignite uses indexes not only for index lookups, but also for exploiting
> > > it's sortedness. In this case materialized view's approach can show
> some
> > > advantages. Let's consider the example:
> > >
> > > SELECT * FROM foo WHERE a > 100 ORDER BY b;
> > >
> > > where both fields 'a' and 'b' are indexed. In this case we will have
> two
> > > alternatives of the query execution:
> > >
> > > 1. Use index 'a' for index conditioned scan and then sort rows by 'b'
> > > 2. Use index scan over 'b' and then apply filter over 'a' - here we can
> > > avoid sorting, because index over 'b' is already sorted.
> > >
> > > If I understand the approach with FilterTableScan2IndexTableScanRule
> > > correctly, at this step the rule should make a decision about which
> > > index to use. It can be very tricky because the rule should consider
> not
> > > only filters, but also collations. This leads to increasing the
> > > complexity of such rules. With materialized views approach we just
> > > register all indexes with their caollation and let the cost model do
> its
> > > job. Our rules are very simple. The complexity is encapsulated in the
> > > index scan cost estimation.
> > >
> > > As for your example with disjunctive predicate:
> > >
> > > > SELECT * FROM foo WHERE a > 100 or b < 1000;
> > >
> > > I think with 

Re: Using indexes rather than table scans with Calcite

2020-05-31 Thread Vladimir Ozerov
Hi Roman,

This heavily depends on the architecture of the planner. In Hazelcast we
have separate logical and physical phases. The goal of logical phase is
normalization of a relational tree. In this case your example is converted
to:

LogicalJoin
  LogicalConstrainedScan(A, c>100)
  LogicalScan(B)

Next, during physical planning, different implementations are considered.
For example, if for table A there are many indexes - sorted(a), hash(b),
sorted(c) - then It is possible to prune unnecessary access methods. E.g.,
hash(b) is not considered because it doesn’t add any interesting physical
property, and is costlier than other methods. At the same time, sorted(a)
is considered still, even though it has higher cost than sorted(c), because
it provides an interesting collation.

That is the key difference to materialized views - indexes are considered
as needed.

вс, 31 мая 2020 г. в 22:46, Roman Kondakov :

> Hi Vladimir,
>
> thank you for sharing your point. Could you please clarify some details
> with a rulse-based index selection? You said
>
> > the fundamental problem with "indexes as materialized
> > views" approach is that you have to register them beforehand, instead of
> > using them only when needed.
>
> I agree, it's kind of a problem. What is not clear for me with
> IndexScanRule-based approach is how to decide when and which index we
> need? I understand that is is pretty easy to do in the case like this:
>
> Filter
>   Scan
>
> we can match the IndexScanRule on this pattern and do an index lookup
> using filter condition. But what to do in the more complex scenarios?
> Let's consider an example
>
> SELECT * FROM A JOIN B ON A.a=B.b WHERE A.c > 100
>
> where A.a, A.c and B.b are indexed fields. The logical plan for this
> query might look like this:
>
> LogicalJoin(A.a=B.b)
>   LogicalFilter(A.c > 100)
> LogicalScan(A)
>   LogicalScan(B)
>
> as I understand (please correct me if I'm wrong), with the rule-based
> approach, after allpying IndexScanRule the plan will look like this:
>
> LogicalJoin(A.a=B.b)
>   PhysicalIndexScan(A.c, lower bound = 100)
>   PhysicalTableScan(B)
>
> But in this case we lose the possibility of using index scans over A.a
> and B.b and joining them with MergeJoin, which can be more efficient
> plan in terms of the cost.
>
> My question is: how rule-based approach handle this scenario? Will it
> re-apply IndexScanRule once again to produce PhysicalIndexScan(A.a) and
> PhysicalIndexScan(B.b)? Or am I missing the crucial point of a rule-base
> approach?
>
> Thank you in advance!
>
>
> --
> Kind Regards
> Roman Kondakov
>
>
> On 31.05.2020 21:39, Vladimir Ozerov wrote:
> > As already mentioned, the fundamental problem with "indexes as
> materialized
> > views" approach is that you have to register them beforehand, instead of
> > using them only when needed. On the other hand, the complexity of index
> > planning comes from cost estimation and predicate splitting.
> > Materializations cannot help you with that anyhow. This is why I call
> this
> > approach (not materialized views per se) "hacky" - you reuse several
> simple
> > parts of the Calcite infrastructure at the cost of loss in the
> flexibility
> > of the planner, while the most complicated parts still need to be
> > implemented by hand.
> >
> > Materialized views could be a good fit e.g for partial indexes because in
> > this case, Calcite could help you with complex subsumption mechanics. But
> > for standard indexes, the pros/cons balance is not that obvious.
> >
> > вс, 31 мая 2020 г. в 19:28, xu :
> >
> >> Hi Tim,
> >>
> >> I am working on MySQL InnoDB adapter and trying to introduce this to
> >> Calcite, currently it is only in early stage, and not approved/reviewed
> by
> >> committers yet. Anyway, we are facing the same problem like what index
> to
> >> use, how to push down order by operation, etc. I have developed a simple
> >> rule based adapter to be "index aware" and being able to leverage a
> MySQL
> >> InnoDB storage engine written in Java. Hope this will help you to
> explore
> >> more options.
> >>
> >> https://issues.apache.org/jira/browse/CALCITE-4034
> >>
> >> Thanks,
> >> Xu
> >>
> >> Haisheng Yuan  于2020年5月31日周日 下午10:06写道:
> >>
> >>> Hi Roman,
> >>>
> >>> Thank you for sharing your thoughts.
> >>>
> >>>> It can be very tricky because the rule should consider not
> >>>> only filters, but also collations. This 

Re: Using indexes rather than table scans with Calcite

2020-06-02 Thread Vladimir Ozerov
ombined in more
> > ways than regular indexes. In this case, and many others, it is worth
> > thinking about the problem at a high level. For example, combining two
> > bitmap indexes can be modeled as a join, where the join keys are the
> > record ids, and the record ids are sorted within each index value.
> > Thinking at the high level can find plans that the rule-based approach
> > will never find.
> >
> > Indexes-as-MVs, indexes-as-tables, and
> > index-filtered-table-scan-as-join are other examples of the high-level
> > approach.
> >
> > Julian
> >
> >
> >
> > On Mon, Jun 1, 2020 at 12:00 PM Roman Kondakov
> >  wrote:
> > >
> > > Hi Xiening,
> > >
> > > the example was synthetic. What is still not clear for me is how to
> > > exploit index sortedness with a rule based approach. As far as I
> > > understand with this approach we need to write complex rules (for
> > > example [1]) that should decide which index is useful and which is not
> > > useful. These rules should take into account both statistics and
> > > collations, so they do some part of work that should be done by a cost
> > > model. And it makes writing such rules quite a difficult task.
> > >
> > > With a materialized views approach we can register all indexes as
> scans,
> > > push filters to them if needed. And the cost model, not a rule, will
> > > decide which index is better based on its cost and output collation.
> > >
> > > So the benefits of rule based approach are not so obvious to me. I
> would
> > > really appreciate if you could tell me in what cases rule-based
> approach
> > > is better. I understand that its definitely better in scenarios when
> the
> > > number of indexes is very high. But may be there are some other
> advantages?
> > >
> > > Thank you!
> > >
> > > [1]
> > >
> https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/rules/DbScanToIndexScanPrule.java
> > >
> > > --
> > > Kind Regards
> > > Roman Kondakov
> > >
> > >
> > > On 01.06.2020 21:00, Xiening Dai wrote:
> > > > Hi Roman,
> > > >
> > > > The example you mentioned is an advanced scenario. Note that there
> are different types of index, such as clustered index, secondary index,
> covered and non-covered index. In your case, typical OLTP/OLAP optimizer
> would create an index-based join on top of the range table scan (or
> FilteredTableScan in your term). And these transformation can definitely be
> based on rules. But the difficult part is actually the statistics
> estimation and cost calculation. You could end up with higher runtime cost
> with index based join when join cardinality is high.
> > > >
> > > > But back to the original question, if we’d like to leverage index on
> table scan, I think simple rule would serve the purpose. In fact, we have
> FilterTableScanPredicatePushdownRule in our system which does exactly the
> same thing.
> > > >
> > > >> On May 31, 2020, at 12:45 PM, Roman Kondakov
>  wrote:
> > > >>
> > > >> Hi Vladimir,
> > > >>
> > > >> thank you for sharing your point. Could you please clarify some
> details
> > > >> with a rulse-based index selection? You said
> > > >>
> > > >>> the fundamental problem with "indexes as materialized
> > > >>> views" approach is that you have to register them beforehand,
> instead of
> > > >>> using them only when needed.
> > > >>
> > > >> I agree, it's kind of a problem. What is not clear for me with
> > > >> IndexScanRule-based approach is how to decide when and which index
> we
> > > >> need? I understand that is is pretty easy to do in the case like
> this:
> > > >>
> > > >> Filter
> > > >>  Scan
> > > >>
> > > >> we can match the IndexScanRule on this pattern and do an index
> lookup
> > > >> using filter condition. But what to do in the more complex
> scenarios?
> > > >> Let's consider an example
> > > >>
> > > >> SELECT * FROM A JOIN B ON A.a=B.b WHERE A.c > 100
> > > >>
> > > >> where A.a, A.c and B.b are indexed fields. The logical plan for this
> > > >> query might look like this:
> > > >>
> > > 

How to support case-insensitive identifier resolution?

2020-06-16 Thread Vladimir Ozerov
Hi colleagues,

I am trying to implement case insensitive resolution of column/table/schema
names for our Apache Calcite integration in Hazelcast and got stuck. I hope
that the community might help me.

Consider that we have a table source that contains the following Java class:

class Employee {
LocalDate birthDate
}

I would like to have an SQL engine that could resolve identifiers in
accordance with ANSI SQL standard, which basically says that unquoted
identifiers should be compared in a case-insensitive way, while quoted
identifiers should be compared in a case-sensitive way. Let's focus on
columns only for now:

SELECT birthDate FROM employee // OK
SELECT birthdate FROM employee // OK
SELECT BIRTHDATE FROM employee // OK
SELECT `birthDate` FROM employee // OK
SELECT `birthdate` FROM employee // Fail
SELECT `BIRTHDATE` FROM employee // Fail

That is, my source is a collection of Java objects, and the natural name of
the column is "birthDate". But I would like it to be accessible as
"birthDate", birthDate, BIRTHDate, etc.

My problem comes from the fact that casing configuration is applied during
parsing and by the time the table is asked for a column, the information
whether the user request was quoted or not is lost. Consider that I have a
table RelDataType["birthDate"]. Now consider what happens with different
combinations of casing configuration:
1) [unquoted=UNCHANGED, quoted=UNCHANGED]: "SELECT BIRTHDATE" doesn't work
obviously
2) [unquoted=UPPER, quoted=UNCHANGED]: "SELECT BIRTHDATE" doesn't work
again, because parser normalizes unqouted identifier to upper case, but
RelDataType has a column "birthDate"
3) Same as p.2, but with manual normalization of RelDataType
to RelDataType["BIRTHDATE"]: "SELECT BIRTHDATE" works now, but "SELECT
`birthDate`" don't!

Is there any built-in solution to the above problem?

Regards,
Vladimir.


RelFieldTrimmer misses optimization opportunity for always false condition

2020-08-11 Thread Vladimir Ozerov
Hi colleagues,

Consider the following query. If the RelFieldTrimmer is applied to this,
then the expression is reduced to an empty LogicalValues:
SELECT field FROM table WHERE TRUE IS FALSE

However, the following query will not be simplified, leaving a table scan
with "always false" filter:
SELECT * FROM table WHERE TRUE IS FALSE

After some debugging I found that the problem is in the following piece of
code in the RelFieldTrimmer:

// If the input is unchanged, and we need to project all columns,
// there's nothing we can do.
if (newInput == input
&& fieldsUsed.cardinality() == fieldCount) {
  return result(filter, Mappings.createIdentity(fieldCount));
}

My question - is it a known issue? Looks like this early return from the
method misses an important optimization opportunity.  Can this check be
removed completely?

Regards,
Vladimir


Decouple core from linq4j and Avatica

2020-11-24 Thread Vladimir Ozerov
Hi colleagues,

Many Calcite integrations use only part of the framework. Specifically, it
is common to use only the parser/optimizer part. JDBC and runtime are used
less frequently because they are not very well suited for mature processing
engines (e.g. Enumerable runs out of memory easily).

However, in order to use the parser/optimizer from the core module, you
also need to add "linq4j" and Avatica modules to the classpath, which is
not convenient - why to include modules, that you do not use?

It turns out that most of the dependencies are indeed leaky abstractions,
that could be decoupled easily. For example, the RelOptUtil class from the
"core" depends on ... two string constants from the Avatica module.

I would like to propose to decouple the "core" module from "ling4j" and
Avatica. For example, we may introduce the new "common" module, that will
hold common constants, utility classes, and interfaces (e.g. Meta). Then,
we can organize the dependencies like this:
common -> core
common -> linq4j
common -> Avatica

Finally, we may shade and relocate the "common" module into the "core"
during the build. In the end, we will have -2 runtime dependencies with
relatively little effort. In principle, the same approach could be applied
to Janino and Jackson dependencies, but it could be more complex, so my
proposal is only about "linq4" and Avatica.

How do you feel about it? Does this proposal sense to the community? If
yes, I can try implementing the POC for this.

Regards,
Vladimir.


Re: Decouple core from linq4j and Avatica

2020-11-27 Thread Vladimir Ozerov
Hi colleagues,

Thank you for the valuable feedback. The problem is indeed complex. I share
the worry that complete decoupling might be too disruptive for users, since
they will observe compilation problems when migrating to the newer version,
and will have to update their dependencies, which also could be problematic
(e.g. due to security concerns). So I'd like to propose a slightly
different approach that should not cause any problems for the existing
users. We change the goal from the complete decoupling to the *isolation *of
dependent classes.

Let me explain it with Avatica as an example. There are two class of
Avatica-related dependencies in the core: (1) utilities (e.g. classes from
org.apache.calcite.avatica.util), and (2) logic (e.g. classes from
org.apache.calcite.jdbc, org.apache.calcite.adapter.jdbc). The first class
is very easy to eliminate. The second class cannot be eliminated with the
serious repackaging of the whole Calcite. So we can do the following:

1. Introduce the "commons" module, and move utilities there, thus solving
(1).
2. Shade the "commons" module into the "core" during the build - if we do
this, the existing users will not have to change their dependencies, so
this is a critically important step (at least for now). An alternative to
this is just to copy-paste utility classes into the "core" module,
violating DRY
3. Contain the outstanding Avatica dependencies to a couple of JDBC-related
packages, and add a static analysis rule to disallow Avatica classes in any
other package. This may require some advanced refactoring (e.g.
CalciteConnectionConfig)

As a result, Avatica dependency is reduced to a handful of packages, and
existing applications will work mostly seamlessly during migration. Now we
can do one of two things:
1. Either create a separate reduced artifact "core-reduced" without
Avatica-dependent packages
2. Since many products shade Calcite during the build, we can advise them
to just exclude Avatica-dependent packages when shading

How does it sound?

Regards,
Vladimir


ср, 25 нояб. 2020 г. в 10:48, Chunwei Lei :

> I like the idea. But I have the same worry as Haisheng.
>
>
> Best,
> Chunwei
>
>
> On Wed, Nov 25, 2020 at 3:07 PM Xin Wang  wrote:
>
> > +1 for this idea. We only use the parser/optimizer part.
> >
> > JiaTao Tao  于2020年11月25日周三 下午2:38写道:
> >
> > > +1 for this idea, I have been developing Calcite for a long
> time(counting
> > > during project Kylin), we all treat calcite as an optimizer, but we
> need
> > to
> > > consider overhead.
> > >
> > > I aggre with Stamatis: "since those dependencies were not causing any
> > real
> > > trouble."
> > >
> > >
> > > What really troubling me is that when we do some in logical, we may
> have
> > to
> > > consider the implemnt, for an example, we used keep "In", not convert
> to
> > > join or "OR", but calcite have no impl about "In".
> > >
> > >
> > > Regards!
> > >
> > > Aron Tao
> > >
> > >
> > >
> > > Haisheng Yuan  于2020年11月25日周三 下午12:57写道:
> > >
> > > > > I would like to propose to decouple the "core" module from "ling4j"
> > and
> > > > Avatica.
> > > > I like the idea.
> > > >
> > > > Moving Enumerable out of core may be time consuming and disruptive,
> > > > because many core tests are using Enumerable to verify plan quality
> and
> > > > correctness.
> > > >
> > > > Best,
> > > > Haisheng
> > > >
> > > > On 2020/11/24 23:42:19, Stamatis Zampetakis 
> wrote:
> > > > > Hi Vladimir,
> > > > >
> > > > > Personally, I like the idea.
> > > > > I had similar thoughts in the past but it didn't try to break it
> down
> > > > since
> > > > > those dependencies were not causing any real trouble.
> > > > >
> > > > > Let's see what the others think.
> > > > >
> > > > > Best,
> > > > > Stamatis
> > > > >
> > > > >
> > > > > On Tue, Nov 24, 2020 at 7:30 PM Vladimir Ozerov <
> ppoze...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hi colleagues,
> > > > > >
> > > > > > Many Calcite integrations use only part of the framework.
> > > > Specifically, it
> > > > > > is common to use only the parser/optimizer part. JDBC and runtime
> > are
> &

Re: Need a Jason output if physical tree

2020-12-18 Thread Vladimir Ozerov
Hi Liya,

I do not think, this is a global decision. If you go bottom-up in the plan,
and find the Project operator, than none parent operators will use any
columns from the lower scans, other than those present in the Project.

It is a common practice for Calcite integrations to restrict the number of
columns returned from scans. It is achieved as follows:
1) There should be rules to extract projections from some operators (e.g.
Joins)
2) There should be rules to transpose projects with child operators
3) Finally, you should create a rule to move project into scan for the
pattern [Project<-Scan]

When all three prerequisites are ready, the optimizer will produce missing
projects (if any), then push them down the operator tree, and finally merge
into the scan.

Regards,
Vladimir

Пт, 18 дек. 2020 г. в 13:34, Fan Liya :

> Hi Bhavya,
>
> Thanks for your explanation.
>
> I do not think a rule like ProjectFilterTableScan solves the problem.
> To see this, please note that the parent operator of the LogicalProject may
> need other columns from the table
> (e.g. the parent operator may be a join, which uses some other columns as
> the join key).
>
> So the column pruning of table scan should be a global decision, depending
> on the whole plan.
>
> Calcite has RelMdColumnOrigins, which can help in this scenario.
> In particular, given an output column in the output operator, it gives the
> dependent columns in the underlying source tables.
> So columns not referenced from the final output can be pruned from the
> source table.
>
> Best,
> Liya Fan
>
>
> On Fri, Dec 18, 2020 at 4:43 PM Bhavya Aggarwal 
> wrote:
>
> > Hi Liya,
> >
> > Please see below , I have a query as given below
> >
> > SELECT t_1.ID FROM USERS t_1 where NAME = 'HELLO';
> >
> > Now I know that I have to fetch two columns from my columnar database
> i.e.
> > ID, Name, so need to pass these parameters in my custom Table Scan.
> > But when I convert the LogicalTableScan to my custom Table scan I do not
> > have that information, please see the tree below, in the Logical Table
> Scan
> > the Input is null, I was hoping that I will have the columns in the
> > LogicalTableScan, The question is what do I need to do in Calcite to pass
> > that information to the LogicalTableScan or in my physical operator. The
> > way I have found is that I create a Rule like ProjectFilterTableScan rule
> > and then extract what information I need. Is there a better way to do
> this?
> >
> > -Logical JSON Plan 
> > > {
> > >   "rels": [
> > > {
> > >   "id": "0",
> > >   "relOp": "LogicalTableScan",
> > >   "table": [
> > > "USERS"
> > >   ],
> > >   "inputs": []
> > > },
> > > {
> > >   "id": "1",
> > >   "relOp": "LogicalFilter",
> > >   "condition": {
> > > "op": {
> > >   "name": "=",
> > >   "kind": "EQUALS",
> > >   "syntax": "BINARY"
> > > },
> > > "operands": [
> > >   {
> > > "op": {
> > >   "name": "CAST",
> > >   "kind": "CAST",
> > >   "syntax": "SPECIAL"
> > > },
> > > "operands": [
> > >   {
> > > "input": 1,
> > > "name": "$1"
> > >   }
> > > ],
> > > "type": {
> > >   "type": "CHAR",
> > >   "nullable": true,
> > >   "precision": 5
> > > }
> > >   },
> > >   {
> > > "literal": "HELLO",
> > > "type": {
> > >   "type": "CHAR",
> > >   "nullable": false,
> > >   "precision": 5
> > > }
> > >   }
> > > ]
> > >   }
> > > },
> > > {
> > >   "id": "2",
> > >   "relOp": "LogicalProject",
> > >   "fields": [
> > > "ID"
> > >   ],
> > >   "exprs": [
> > > {
> > >   "input": 0,
> > >   "name": "$0"
> > > }
> > >   ]
> > > }
> > >   ]
> > > }
> > >
> > >> ---
> > >
> > >
> > Thanks and regards
> > Bhavya
> >
> > On Fri, Dec 18, 2020 at 11:54 AM Fan Liya  wrote:
> >
> > > Hi Bhavya,
> > >
> > > Sorry I do not understand your question. Why is it difficult to pass
> sth.
> > > to child operators?
> > >
> > > Best,
> > > Liya Fan
> > >
> > >
> > > On Fri, Dec 18, 2020 at 12:38 PM Bhavya Aggarwal 
> > > wrote:
> > >
> > > > Hi Liya,
> > > >
> > > > Actually the question is how to pass the projection columns to Table
> > Scan
> > > > as right now in the LogicalTableScan there are no projection columns
> > > being
> > > > passed so when I am trying to create my custom JSON , I do not have
> > those
> > > > projected columns or columns that are being used in the query. I want
> > to
> > > > understand what is the calcite mechanism for passing it to child
> > > operators
> > > > without using Rules as

Re: Need a Jason output if physical tree

2020-12-21 Thread Vladimir Ozerov
Hi Liya,

This will not work AFAIK. Consider the query "SELECT a FROM t WHERE b>1".
The top-level operator has only one column:

Root[$0]
  Project[$0]
Filter[$1>1]
  Scan[$0=a, $1=b]

If you invoke RelMdColumnOrigins on Root[$0], you will get [t.a], but miss
[t.b].
To my knowledge, rules are the only way to reliably. constrain columns
returned from the scan.

Regards,
Vladimir.

вт, 22 дек. 2020 г. в 05:14, Fan Liya :

> Hi Bhavya,
>
> Sorry I am not sure if I fully understand your question.
>
> Let me try to answer it according to my understanding:
>
> 1. Through RelColumnMDOrigins, we can get the RelColumnOrigin object, which
> includes a RelOptTable object.
> 2. The table scan also has a RelOptTable object, and all table scans of the
> plan can be found (e.g. through a visitor)
> 3. With the information of 1 and 2, given any output column, we can get to
> know it is derived from which columns from which table scans.
> 4. With the information of 3, given a table scan, we can get to know which
> column is never used in any output columns, and such columns can be pruned.
>
> Best,
> Liya Fan
>
>
> On Mon, Dec 21, 2020 at 11:31 PM Bhavya Aggarwal 
> wrote:
>
> > Hi Liya,
> >
> > I had a look at the RelColumnMDOrigins and it is useful in determining
> > which columns are from which table but still I am not sure how can I get
> > the column information for TableScan without the rules. If you have any
> > specific example where we have used this approach will be really helpful
> to
> > me.
> >
> > Thanks and regards
> > Bhavya
> >
> > On Mon, Dec 21, 2020 at 5:53 PM Fan Liya  wrote:
> >
> > > Hi Bhavya,
> > >
> > > IMO, to solve the problem from a global view, the following steps needs
> > to
> > > be taken:
> > >
> > > 1. Generate a physical plan in the original way (without considering
> > column
> > > pruning in the table scan)
> > > 2. Modify all the table scans in the plan with the RelColumnMDOrigins
> > > utility (the details have been described above)
> > > 3. Post process the plan with one of the following ways:
> > > a) a plan visitor that adjusts other operators in the tree.
> > > b) a light-weight planner (Hep or Volcano with limited rule sets)
> > >
> > > Run the query with the finally generated plan.
> > >
> > > Best,
> > > Liya Fan
> > >
> > >
> > > On Mon, Dec 21, 2020 at 3:33 PM Bhavya Aggarwal 
> > > wrote:
> > >
> > > > Hi Fan,
> > > >
> > > > I looked at the class RelColumnMDOrigins and it is giving me the
> origin
> > > of
> > > > the column, but even if I want to take it as a global decision I am
> not
> > > > sure how to proceed. Can you please elaborate on how to achieve this
> ?
> > I
> > > am
> > > > literally stuck as I do not want to use so many rules as in any case
> I
> > > have
> > > > to pass these to the TableScan, even if the user does a select * from
> > > > table, I need to add all those columns to the table scan.
> > > >
> > > > Regards
> > > > Bhavya
> > > >
> > > > --
> > > > Your feedback matters - At Knoldus we aim to be very professional in
> > our
> > > > quality of work, commitment to results, and proactive communication.
> If
> > > > you
> > > > feel otherwise please share your feedback
> > > >  and we would work on it.
> > > >
> > >
> >
> >
> > --
> > *Bhavya Aggarwal*
> > CTO & Partner
> > Knoldus Inc. 
> > +91-9910483067
> > Canada - USA - India - Singapore
> >   >
> >  
> >
> > --
> > Your feedback matters - At Knoldus we aim to be very professional in our
> > quality of work, commitment to results, and proactive communication. If
> > you
> > feel otherwise please share your feedback
> >  and we would work on it.
> >
>


Contributor rights

2021-01-09 Thread Vladimir Ozerov
Hi,

Could you please grant me contributor rights in Calcite JIRA? My username
is "vozerov".

Thank you.
Vladimir.


Re: Contributor rights

2021-01-11 Thread Vladimir Ozerov
Thank you very much, Francis!

вс, 10 янв. 2021 г. в 00:45, Francis Chuang :

> Hey Vladimir,
>
> I've added you to the contributor role in jira.
>
> Francis
>
> On 9/01/2021 8:47 pm, Vladimir Ozerov wrote:
> > Hi,
> >
> > Could you please grant me contributor rights in Calcite JIRA? My username
> > is "vozerov".
> >
> > Thank you.
> > Vladimir.
> >
>


Re: [DISCUSS] Apache Calcite Online Meetup January 2021

2021-01-11 Thread Vladimir Ozerov
Hi,

I can share our experience with Apache Calcite  integration into Hazelcast
distributed SQL engine.

Regards,
Vladimir

Вт, 5 янв. 2021 г. в 00:48, Vineet G :

> Hi Stamatis,
>
> Something has come up and unfortunately I will not be able to present the
> talk.
>
> Vineet
>
> > On Jan 3, 2021, at 1:37 PM, Stamatis Zampetakis 
> wrote:
> >
> > I updated the agenda on meetup to include Julian's talk around spatial
> > queries.
> >
> > So far we have four presentations lasting approximately 1h45 so I still
> > find the duration reasonable.
> >
> > Of course if there are more people interested to present something we can
> > schedule another meetup in April as Julian suggested.
> > I am always happy to see what other people are working on and more
> Calcite
> > use-cases.
> >
> > Best,
> > Stamatis
> >
> > On Sun, Jan 3, 2021 at 2:09 AM Julian Hyde 
> wrote:
> >
> >> In other news I’ll be co-presenting (with Mosha Pasumansky) a talk
> >> “Open source SQL - beyond parsers: ZetaSQL and Apache Calcite” at the
> >> Northwest Database Society Annual Meeting on January 29th. It’s virtual
> and
> >> free, but you must sign up to attend.
> >>
> >> Julian
> >>
> >> [1] https://sites.google.com/view/nwds2021
> >>
> >>> On Jan 2, 2021, at 12:47 PM, Julian Hyde 
> wrote:
> >>>
> >>> I can give a talk “Implementing spatial queries using algebra
> >> rewrites”, 20 minutes.
> >>>
> >>> But if that makes the meetup too long, I am equally happy to postpone
> >> the talk. How about scheduling another meetup  in say April?
> >>>
> >>> Julian
> >>>
>  On Dec 31, 2020, at 3:10 AM, Stamatis Zampetakis 
> >> wrote:
> 
>  I just published the event on Meetup [1].
> 
>  The agenda is not yet finalized so if there are people who would like
> to
>  give a talk or add/remove things from the agenda please reply to this
>  thread.
> 
>  Best,
>  Stamatis
> 
>  [1] https://www.meetup.com/Apache-Calcite/events/275461117/
> 
> >> On Mon, Nov 30, 2020 at 12:37 AM Rui Wang 
> >> wrote:
> >
> > Title: event timestamp semantic based streaming SQL
> > Abstract: this talk will cover in Calcite Streaming SQL case, how to
> >> reason
> > data completeness in terms of event timestamp semantic and how to
> >> control
> > materialization latency given unbounded input data (in Calcite
> roadmap
> >> but
> > not implemented yet).
> >
> > Duration: 20~30 mins
> >
> >> On Tue, Nov 24, 2020 at 8:56 AM Slim Bouguerra 
> >> wrote:
> >>
> >> this is a great idea thanks @Statmatis looking forward to learning
> >> more
> >> about Calcite especially the Streaming work.
> >>
> >>> On Mon, Nov 23, 2020 at 2:19 PM Rui Wang 
> >> wrote:
> >>
> >>> Sorry for the late reply Statmatis. I have recently been pretty
> busy
> >> on
> >>> work as it is approaching the end of the year.
> >>>
> >>> The time in [1] works perfectly for me. I will share the abstract
> and
> >>> expected duration soon (should within this week).
> >>>
> >>>
> >>> -Rui
> >>>
> >>> On Fri, Nov 20, 2020 at 2:11 AM Stamatis Zampetakis <
> >> zabe...@gmail.com
> >>
> >>> wrote:
> >>>
>  That would be great Vineet!
> 
>  @Julian, @Rui, @Vineet:
>  Can you share a small abstract (2-3 sentences) and expected
> >> duration?
>  Can you check if the date/times proposed previously [1] work for
> >> you.
> >> If
>  not feel free to propose another slot.
> 
>  Best,
>  Stamatis
> 
>  [1] https://s.apache.org/uhrzo
> 
>  On Thu, Nov 19, 2020 at 6:18 PM Vineet Garg 
> > wrote:
> 
> > I think this is a great idea. +1 for the online meetup.
> >
> > If there are slots left I can also talk about how Hive leverages
> >>> Calcite
>  to
> > do query optimization.
> >
> > -Vineet
> >
> > On Fri, Nov 6, 2020 at 7:21 AM Stamatis Zampetakis <
> >> zabe...@gmail.com>
> > wrote:
> >
> >> Let's try to fix the date/time and tentative agenda so that we
> > can
> >>> add
> > some
> >> information on meetup [1].
> >>
> >> So far we have three presenters, Julian, Rui, and myself. We can
> >>> start
> > like
> >> that and if in the process there are more people interested to
> >> give a
> > small
> >> talk we can update the program.
> >>
> >> Let's try to get a date in the last two weeks of January to give
> >> us a
>  bit
> >> more time to prepare. Personally, I don't have a preference for
> >> that
> > being
> >> a business day or not and I am in UTC+1.
> >> For instance, how do you feel about Wednesday, 20 January 2021,
>  18:00:00
> > to
> >> 21:00 UTC+1 [2] ?
> >>

Re: [DISCUSS] Apache Calcite Online Meetup January 2021

2021-01-12 Thread Vladimir Ozerov
Hi Stamatis,

Sorry, my previous email was not clear enough. To clarify - I was thinking
about the mentioned next meetup in April because the schedule for the
January meetup appears to be too packed already (3 talks in 1:30h). I
I am ready to talk in both January and April. Just want to ensure that we
do not overwhelm our attendees.

I'll provide the talk details in the nearest day.

Regards,
Vladimir.

вт, 12 янв. 2021 г. в 01:41, Stamatis Zampetakis :

> That would be great Vladimir, I will update the agenda.
>
> If possible please provide a title, duration, and abstract.
>
> Best,
> Stamatis
>
> On Mon, Jan 11, 2021 at 8:35 PM Vladimir Ozerov 
> wrote:
>
> > Hi,
> >
> > I can share our experience with Apache Calcite  integration into
> Hazelcast
> > distributed SQL engine.
> >
> > Regards,
> > Vladimir
> >
> > Вт, 5 янв. 2021 г. в 00:48, Vineet G :
> >
> > > Hi Stamatis,
> > >
> > > Something has come up and unfortunately I will not be able to present
> the
> > > talk.
> > >
> > > Vineet
> > >
> > > > On Jan 3, 2021, at 1:37 PM, Stamatis Zampetakis 
> > > wrote:
> > > >
> > > > I updated the agenda on meetup to include Julian's talk around
> spatial
> > > > queries.
> > > >
> > > > So far we have four presentations lasting approximately 1h45 so I
> still
> > > > find the duration reasonable.
> > > >
> > > > Of course if there are more people interested to present something we
> > can
> > > > schedule another meetup in April as Julian suggested.
> > > > I am always happy to see what other people are working on and more
> > > Calcite
> > > > use-cases.
> > > >
> > > > Best,
> > > > Stamatis
> > > >
> > > > On Sun, Jan 3, 2021 at 2:09 AM Julian Hyde 
> > > wrote:
> > > >
> > > >> In other news I’ll be co-presenting (with Mosha Pasumansky) a
> talk
> > > >> “Open source SQL - beyond parsers: ZetaSQL and Apache Calcite” at
> the
> > > >> Northwest Database Society Annual Meeting on January 29th. It’s
> > virtual
> > > and
> > > >> free, but you must sign up to attend.
> > > >>
> > > >> Julian
> > > >>
> > > >> [1] https://sites.google.com/view/nwds2021
> > > >>
> > > >>> On Jan 2, 2021, at 12:47 PM, Julian Hyde 
> > > wrote:
> > > >>>
> > > >>> I can give a talk “Implementing spatial queries using algebra
> > > >> rewrites”, 20 minutes.
> > > >>>
> > > >>> But if that makes the meetup too long, I am equally happy to
> postpone
> > > >> the talk. How about scheduling another meetup  in say April?
> > > >>>
> > > >>> Julian
> > > >>>
> > > >>>> On Dec 31, 2020, at 3:10 AM, Stamatis Zampetakis <
> zabe...@gmail.com
> > >
> > > >> wrote:
> > > >>>>
> > > >>>> I just published the event on Meetup [1].
> > > >>>>
> > > >>>> The agenda is not yet finalized so if there are people who would
> > like
> > > to
> > > >>>> give a talk or add/remove things from the agenda please reply to
> > this
> > > >>>> thread.
> > > >>>>
> > > >>>> Best,
> > > >>>> Stamatis
> > > >>>>
> > > >>>> [1] https://www.meetup.com/Apache-Calcite/events/275461117/
> > > >>>>
> > > >>>>>> On Mon, Nov 30, 2020 at 12:37 AM Rui Wang  >
> > > >> wrote:
> > > >>>>>
> > > >>>>> Title: event timestamp semantic based streaming SQL
> > > >>>>> Abstract: this talk will cover in Calcite Streaming SQL case, how
> > to
> > > >> reason
> > > >>>>> data completeness in terms of event timestamp semantic and how to
> > > >> control
> > > >>>>> materialization latency given unbounded input data (in Calcite
> > > roadmap
> > > >> but
> > > >>>>> not implemented yet).
> > > >>>>>
> > > >>>>> Duration: 20~30 mins
> > > >>>>>
> > > >>>>>> O

Re: [DISCUSS] Apache Calcite Online Meetup January 2021

2021-01-12 Thread Vladimir Ozerov
Hi Stamatis, Julian,

Thank you for releasing the slot. Just to be crystal clear - I was very
late with my proposal, and have no urge to present in January. Both January
and April are perfectly fine with me. So please prioritize this talk over
others only if you see a really good reason to do so. In any case, I
confirm that I can present in January. Please find the talk details below.

*Speaker:* Vladimir Ozerov
*Title:* Apache Calcite integration in Hazelcast IMDG
*Abstract: *
Hazelcast IMDG is a distributed in-memory key-value store. In this talk, I
will present how we used Apache Calcite to create a new distributed SQL
engine that queries Hazelcast IMDG data.
We start with motivation and general design. Then we examine how Hazelcast
IMDG leverages Apache Calcite for query parsing, validation, and
optimization, and why we decided not to use Apache Calcite for JDBC and
query execution. We also discuss several Apache Calcite problems that
negatively affect the integration and propose possible future improvements.

Regards,
Vladimir.

ср, 13 янв. 2021 г. в 02:48, Stamatis Zampetakis :

> Yesterday, I updated our website to also include Vladimir's talk about
> Hazelcast so I think it is better to keep it that way.
> Initially I had in mind something between 2 and 4 talks for this meetup
> (2-3h) so I think we are good to go.
>
> Let's now freeze the agenda to avoid changing it till the last minute.
> I will open up the discussion for the next meetup in another email so we
> start filling the slots for April.
>
> It's definitely useful to have the talks on camera. I will record them via
> Zoom and split them after the meetup.
> If the presenters give their consent I can upload them afterwards to
> Youtube/Vimeo.
>
> Best,
> Stamatis
>
>
> On Wed, Jan 13, 2021 at 12:15 AM Julian Hyde  wrote:
>
> > I'm happy to yield my spot (and speak instead in April) if Vladimir is
> > still prepared to talk. I'd love to get his work on Hazelcast
> > integration on the record!
> >
> > By the way, will it be possible to record these talks and post them
> > *as separate items* to (say) YouTube or Vimeo? I think it's important
> > that they are separate, so that people can find the particular talk
> > they are interested in.
> >
> > Julian
> >
> > On Tue, Jan 12, 2021 at 12:17 AM Vladimir Ozerov 
> > wrote:
> > >
> > > Hi Stamatis,
> > >
> > > Sorry, my previous email was not clear enough. To clarify - I was
> > thinking
> > > about the mentioned next meetup in April because the schedule for the
> > > January meetup appears to be too packed already (3 talks in 1:30h). I
> > > I am ready to talk in both January and April. Just want to ensure that
> we
> > > do not overwhelm our attendees.
> > >
> > > I'll provide the talk details in the nearest day.
> > >
> > > Regards,
> > > Vladimir.
> > >
> > > вт, 12 янв. 2021 г. в 01:41, Stamatis Zampetakis :
> > >
> > > > That would be great Vladimir, I will update the agenda.
> > > >
> > > > If possible please provide a title, duration, and abstract.
> > > >
> > > > Best,
> > > > Stamatis
> > > >
> > > > On Mon, Jan 11, 2021 at 8:35 PM Vladimir Ozerov 
> > > > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > I can share our experience with Apache Calcite  integration into
> > > > Hazelcast
> > > > > distributed SQL engine.
> > > > >
> > > > > Regards,
> > > > > Vladimir
> > > > >
> > > > > Вт, 5 янв. 2021 г. в 00:48, Vineet G :
> > > > >
> > > > > > Hi Stamatis,
> > > > > >
> > > > > > Something has come up and unfortunately I will not be able to
> > present
> > > > the
> > > > > > talk.
> > > > > >
> > > > > > Vineet
> > > > > >
> > > > > > > On Jan 3, 2021, at 1:37 PM, Stamatis Zampetakis <
> > zabe...@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > I updated the agenda on meetup to include Julian's talk around
> > > > spatial
> > > > > > > queries.
> > > > > > >
> > > > > > > So far we have four presentations lasting approximately 1h45
> so I
> > > > still
> > > > > > > find the duration reasonable.
> > > >

Re: [DISCUSS] Apache Calcite Online Meetup January 2021

2021-01-12 Thread Vladimir Ozerov
*Talk duration*: 30 min.

ср, 13 янв. 2021 г. в 10:27, Vladimir Ozerov :

> Hi Stamatis, Julian,
>
> Thank you for releasing the slot. Just to be crystal clear - I was very
> late with my proposal, and have no urge to present in January. Both January
> and April are perfectly fine with me. So please prioritize this talk over
> others only if you see a really good reason to do so. In any case, I
> confirm that I can present in January. Please find the talk details below.
>
> *Speaker:* Vladimir Ozerov
> *Title:* Apache Calcite integration in Hazelcast IMDG
> *Abstract: *
> Hazelcast IMDG is a distributed in-memory key-value store. In this talk, I
> will present how we used Apache Calcite to create a new distributed SQL
> engine that queries Hazelcast IMDG data.
> We start with motivation and general design. Then we examine how Hazelcast
> IMDG leverages Apache Calcite for query parsing, validation, and
> optimization, and why we decided not to use Apache Calcite for JDBC and
> query execution. We also discuss several Apache Calcite problems that
> negatively affect the integration and propose possible future improvements.
>
> Regards,
> Vladimir.
>
> ср, 13 янв. 2021 г. в 02:48, Stamatis Zampetakis :
>
>> Yesterday, I updated our website to also include Vladimir's talk about
>> Hazelcast so I think it is better to keep it that way.
>> Initially I had in mind something between 2 and 4 talks for this meetup
>> (2-3h) so I think we are good to go.
>>
>> Let's now freeze the agenda to avoid changing it till the last minute.
>> I will open up the discussion for the next meetup in another email so we
>> start filling the slots for April.
>>
>> It's definitely useful to have the talks on camera. I will record them via
>> Zoom and split them after the meetup.
>> If the presenters give their consent I can upload them afterwards to
>> Youtube/Vimeo.
>>
>> Best,
>> Stamatis
>>
>>
>> On Wed, Jan 13, 2021 at 12:15 AM Julian Hyde  wrote:
>>
>> > I'm happy to yield my spot (and speak instead in April) if Vladimir is
>> > still prepared to talk. I'd love to get his work on Hazelcast
>> > integration on the record!
>> >
>> > By the way, will it be possible to record these talks and post them
>> > *as separate items* to (say) YouTube or Vimeo? I think it's important
>> > that they are separate, so that people can find the particular talk
>> > they are interested in.
>> >
>> > Julian
>> >
>> > On Tue, Jan 12, 2021 at 12:17 AM Vladimir Ozerov 
>> > wrote:
>> > >
>> > > Hi Stamatis,
>> > >
>> > > Sorry, my previous email was not clear enough. To clarify - I was
>> > thinking
>> > > about the mentioned next meetup in April because the schedule for the
>> > > January meetup appears to be too packed already (3 talks in 1:30h). I
>> > > I am ready to talk in both January and April. Just want to ensure
>> that we
>> > > do not overwhelm our attendees.
>> > >
>> > > I'll provide the talk details in the nearest day.
>> > >
>> > > Regards,
>> > > Vladimir.
>> > >
>> > > вт, 12 янв. 2021 г. в 01:41, Stamatis Zampetakis :
>> > >
>> > > > That would be great Vladimir, I will update the agenda.
>> > > >
>> > > > If possible please provide a title, duration, and abstract.
>> > > >
>> > > > Best,
>> > > > Stamatis
>> > > >
>> > > > On Mon, Jan 11, 2021 at 8:35 PM Vladimir Ozerov > >
>> > > > wrote:
>> > > >
>> > > > > Hi,
>> > > > >
>> > > > > I can share our experience with Apache Calcite  integration into
>> > > > Hazelcast
>> > > > > distributed SQL engine.
>> > > > >
>> > > > > Regards,
>> > > > > Vladimir
>> > > > >
>> > > > > Вт, 5 янв. 2021 г. в 00:48, Vineet G :
>> > > > >
>> > > > > > Hi Stamatis,
>> > > > > >
>> > > > > > Something has come up and unfortunately I will not be able to
>> > present
>> > > > the
>> > > > > > talk.
>> > > > > >
>> > > > > > Vineet
>> > > > > >
>> > > > > > > On Jan 3, 2021, at 1:37 PM, Stamatis Zampetakis <
>> >

Add cross-join suppression to JoinAssociateRule

2021-02-12 Thread Vladimir Ozerov
Hi,

Join order planning is an important optimization problem. The widely-used
heuristic is to consider all bushy trees without cross-joins. There is
proof [1] that a pair of commute and associate rules is a complete ruleset
to explore all bushy trees without the cross-joins if the initial join tree
has no cross-joins.

In Apache Calcite, we do not suppress cross-products. As a result, even
simple join topologies, like a chain, cannot be planned in a reasonable
time for more than 6-7 tables. How do you feel if we add an optional
cross-join suppression to the JoinAssociateRule, and possibly
JoinPushThroughJoinRule?

The cross-join suppression is not the only problem with the exhaustive join
planning in Apache Calcite. But at the very least, it is simple to
implement, and it extends the number of tables in a join that could be
planned exhaustively for some topologies by additional 2-5 tables.

Regards,
Vladimir.

[1] https://dl.acm.org/doi/10.14778/2732977.2732997


Re: Add cross-join suppression to JoinAssociateRule

2021-02-14 Thread Vladimir Ozerov
Hi Julian,

First of all, I want to clarify that if we decide to do the cross-join
suppression, we would have to do that for JoinCommuteRule either. I missed
that in the original email.

Now back to your comments. The proposal assumes that we start with a
connected join graph without the cross-products and never generate other
cross-products. This covers many common queries, but not all. First, as you
mentioned, some queries may potentially benefit from cross-products. I do
not have specific queries in mind, but I cannot prove that such queries
don't exist either. Second, we may have cross-products in an otherwise
connected join graph if the join condition is located in the WHERE clause.
For such queries, we may need to do some pre-processing, like filter
pushdown. Therefore, I assume that real-world products would have to do
some preliminary analysis and/or processing to decide whether it is safe to
use the cross-join suppression. This might not be very convenient for new
users, but IMO perfectly fine for production-grade systems because they
typically do complex multi-phase optimization anyway. For example, recall
how VoltDB decides whether to perform the join planning in the physical
phase depending on the number of joins in the query [1].

Regarding cardinality estimations, the main problem is that the built-in
Calcite mechanism is pretty imprecise because they use mainly row count and
magic numbers. For example, in TPC-H queries, the cardinality estimations
easily reach billions and billions of rows. And even if the system has more
advanced cardinality estimators, such as histograms, the estimation errors
are likely to build up quickly still [2]. That said, it could be difficult
to design a robust heuristic to suppress particular joins. The advantage of
a straightforward cross-join suppression heuristic is that it is a good
choice for a good fraction of real-world queries.

To clarify, the proposal is not to make the cross-join suppression the
default behavior. Rather, we may add it as a configuration property to join
planning rules, that could be used by advanced users.

Regards,
Vladimir.

[1]
https://github.com/VoltDB/voltdb/blob/0f2993cb9e1efe7c2c95cf68b83f10903e2697d3/src/frontend/org/voltdb/compiler/PlannerTool.java#L274
[2] https://dl.acm.org/doi/10.14778/2850583.2850594

сб, 13 февр. 2021 г. в 23:05, Julian Hyde :

> Two separate points.
>
> 1. I recall that there is an important class of plans that first forms the
> Cartesian product of some very small relations, and then joins everything
> else to it. Does anyone else encounter these plans? Would these plans be
> unavailable if we made your proposed change?
>
> 2. If I join on a low cardinality attribute (eg customers to suppliers on
> state) the result is almost as bad a cross join. I presume you would want
> to treat this join similarly. If so, it would make sense to have the rules
> on selectivity (or similar) rather than structure alone.
>
> Julian
>
> > On Feb 12, 2021, at 23:04, Vladimir Ozerov  wrote:
> >
> > Hi,
> >
> > Join order planning is an important optimization problem. The widely-used
> > heuristic is to consider all bushy trees without cross-joins. There is
> > proof [1] that a pair of commute and associate rules is a complete
> ruleset
> > to explore all bushy trees without the cross-joins if the initial join
> tree
> > has no cross-joins.
> >
> > In Apache Calcite, we do not suppress cross-products. As a result, even
> > simple join topologies, like a chain, cannot be planned in a reasonable
> > time for more than 6-7 tables. How do you feel if we add an optional
> > cross-join suppression to the JoinAssociateRule, and possibly
> > JoinPushThroughJoinRule?
> >
> > The cross-join suppression is not the only problem with the exhaustive
> join
> > planning in Apache Calcite. But at the very least, it is simple to
> > implement, and it extends the number of tables in a join that could be
> > planned exhaustively for some topologies by additional 2-5 tables.
> >
> > Regards,
> > Vladimir.
> >
> > [1] https://dl.acm.org/doi/10.14778/2732977.2732997
>


Re: Add cross-join suppression to JoinAssociateRule

2021-02-20 Thread Vladimir Ozerov
Thank you for the feedback. I'll try to prototype it for all the affected
rules.

Regards,
Vladimir

чт, 18 февр. 2021 г. в 13:02, Stamatis Zampetakis :

> Hi Vladimir,
>
> Thanks for bringing up this topic.
>
> There are various works who highlight the importance of cartesian products
> for optimality and try to derive techniques for exploring the complete
> search space efficiently.
> Having said that indeed not considering cartesian products is a common &
> popular heuristic for queries with many relations so I find it a good idea
> to give to the users the option to use it or not.
>
> Best,
> Stamatis
>
> On Sun, Feb 14, 2021 at 9:58 AM Vladimir Ozerov 
> wrote:
>
> > Hi Julian,
> >
> > First of all, I want to clarify that if we decide to do the cross-join
> > suppression, we would have to do that for JoinCommuteRule either. I
> missed
> > that in the original email.
> >
> > Now back to your comments. The proposal assumes that we start with a
> > connected join graph without the cross-products and never generate other
> > cross-products. This covers many common queries, but not all. First, as
> you
> > mentioned, some queries may potentially benefit from cross-products. I do
> > not have specific queries in mind, but I cannot prove that such queries
> > don't exist either. Second, we may have cross-products in an otherwise
> > connected join graph if the join condition is located in the WHERE
> clause.
> > For such queries, we may need to do some pre-processing, like filter
> > pushdown. Therefore, I assume that real-world products would have to do
> > some preliminary analysis and/or processing to decide whether it is safe
> to
> > use the cross-join suppression. This might not be very convenient for new
> > users, but IMO perfectly fine for production-grade systems because they
> > typically do complex multi-phase optimization anyway. For example, recall
> > how VoltDB decides whether to perform the join planning in the physical
> > phase depending on the number of joins in the query [1].
> >
> > Regarding cardinality estimations, the main problem is that the built-in
> > Calcite mechanism is pretty imprecise because they use mainly row count
> and
> > magic numbers. For example, in TPC-H queries, the cardinality estimations
> > easily reach billions and billions of rows. And even if the system has
> more
> > advanced cardinality estimators, such as histograms, the estimation
> errors
> > are likely to build up quickly still [2]. That said, it could be
> difficult
> > to design a robust heuristic to suppress particular joins. The advantage
> of
> > a straightforward cross-join suppression heuristic is that it is a good
> > choice for a good fraction of real-world queries.
> >
> > To clarify, the proposal is not to make the cross-join suppression the
> > default behavior. Rather, we may add it as a configuration property to
> join
> > planning rules, that could be used by advanced users.
> >
> > Regards,
> > Vladimir.
> >
> > [1]
> >
> >
> https://github.com/VoltDB/voltdb/blob/0f2993cb9e1efe7c2c95cf68b83f10903e2697d3/src/frontend/org/voltdb/compiler/PlannerTool.java#L274
> > [2] https://dl.acm.org/doi/10.14778/2850583.2850594
> >
> > сб, 13 февр. 2021 г. в 23:05, Julian Hyde :
> >
> > > Two separate points.
> > >
> > > 1. I recall that there is an important class of plans that first forms
> > the
> > > Cartesian product of some very small relations, and then joins
> everything
> > > else to it. Does anyone else encounter these plans? Would these plans
> be
> > > unavailable if we made your proposed change?
> > >
> > > 2. If I join on a low cardinality attribute (eg customers to suppliers
> on
> > > state) the result is almost as bad a cross join. I presume you would
> want
> > > to treat this join similarly. If so, it would make sense to have the
> > rules
> > > on selectivity (or similar) rather than structure alone.
> > >
> > > Julian
> > >
> > > > On Feb 12, 2021, at 23:04, Vladimir Ozerov 
> wrote:
> > > >
> > > > Hi,
> > > >
> > > > Join order planning is an important optimization problem. The
> > widely-used
> > > > heuristic is to consider all bushy trees without cross-joins. There
> is
> > > > proof [1] that a pair of commute and associate rules is a complete
> > > ruleset
> > > > to explore all bushy trees without the cross-joins if the initial
> join
&g

Re: Add cross-join suppression to JoinAssociateRule

2021-02-26 Thread Vladimir Ozerov
I created the PR [1]. It adds a property that prunes new join trees if
there is an "always true" condition. Strictly speaking, this is not a
suppression of all possible cross-joins. But such check is cheap and should
work well in practical optimizers assuming that a filter-pushdown is
applied, and joins contain only those conditions, that couldn't be
pushed down.

[1] https://github.com/apache/calcite/pull/2359


сб, 20 февр. 2021 г. в 20:34, Vladimir Ozerov :

> Thank you for the feedback. I'll try to prototype it for all the affected
> rules.
>
> Regards,
> Vladimir
>
> чт, 18 февр. 2021 г. в 13:02, Stamatis Zampetakis :
>
>> Hi Vladimir,
>>
>> Thanks for bringing up this topic.
>>
>> There are various works who highlight the importance of cartesian products
>> for optimality and try to derive techniques for exploring the complete
>> search space efficiently.
>> Having said that indeed not considering cartesian products is a common &
>> popular heuristic for queries with many relations so I find it a good idea
>> to give to the users the option to use it or not.
>>
>> Best,
>> Stamatis
>>
>> On Sun, Feb 14, 2021 at 9:58 AM Vladimir Ozerov 
>> wrote:
>>
>> > Hi Julian,
>> >
>> > First of all, I want to clarify that if we decide to do the cross-join
>> > suppression, we would have to do that for JoinCommuteRule either. I
>> missed
>> > that in the original email.
>> >
>> > Now back to your comments. The proposal assumes that we start with a
>> > connected join graph without the cross-products and never generate other
>> > cross-products. This covers many common queries, but not all. First, as
>> you
>> > mentioned, some queries may potentially benefit from cross-products. I
>> do
>> > not have specific queries in mind, but I cannot prove that such queries
>> > don't exist either. Second, we may have cross-products in an otherwise
>> > connected join graph if the join condition is located in the WHERE
>> clause.
>> > For such queries, we may need to do some pre-processing, like filter
>> > pushdown. Therefore, I assume that real-world products would have to do
>> > some preliminary analysis and/or processing to decide whether it is
>> safe to
>> > use the cross-join suppression. This might not be very convenient for
>> new
>> > users, but IMO perfectly fine for production-grade systems because they
>> > typically do complex multi-phase optimization anyway. For example,
>> recall
>> > how VoltDB decides whether to perform the join planning in the physical
>> > phase depending on the number of joins in the query [1].
>> >
>> > Regarding cardinality estimations, the main problem is that the built-in
>> > Calcite mechanism is pretty imprecise because they use mainly row count
>> and
>> > magic numbers. For example, in TPC-H queries, the cardinality
>> estimations
>> > easily reach billions and billions of rows. And even if the system has
>> more
>> > advanced cardinality estimators, such as histograms, the estimation
>> errors
>> > are likely to build up quickly still [2]. That said, it could be
>> difficult
>> > to design a robust heuristic to suppress particular joins. The
>> advantage of
>> > a straightforward cross-join suppression heuristic is that it is a good
>> > choice for a good fraction of real-world queries.
>> >
>> > To clarify, the proposal is not to make the cross-join suppression the
>> > default behavior. Rather, we may add it as a configuration property to
>> join
>> > planning rules, that could be used by advanced users.
>> >
>> > Regards,
>> > Vladimir.
>> >
>> > [1]
>> >
>> >
>> https://github.com/VoltDB/voltdb/blob/0f2993cb9e1efe7c2c95cf68b83f10903e2697d3/src/frontend/org/voltdb/compiler/PlannerTool.java#L274
>> > [2] https://dl.acm.org/doi/10.14778/2850583.2850594
>> >
>> > сб, 13 февр. 2021 г. в 23:05, Julian Hyde :
>> >
>> > > Two separate points.
>> > >
>> > > 1. I recall that there is an important class of plans that first forms
>> > the
>> > > Cartesian product of some very small relations, and then joins
>> everything
>> > > else to it. Does anyone else encounter these plans? Would these plans
>> be
>> > > unavailable if we made your proposed change?
>> > >
>> > > 2. If I join on a low cardinality attribute

Non-additive costs in heterogeneous engines

2021-02-26 Thread Vladimir Ozerov
Hi,

Several products that utilize Apache Calcite for query optimization might
use multiple execution units to execute physical operators concurrently.
Think of a heterogeneous engine that might split execution between a CPU
and a co-processor (GPU, FGPA, etc), or just a multi-core machine in the
simplest case. In such systems, the cumulative cost of an operator is not
additive. That is, cost(A,B) != cost(A) + cost(B).

Consider a theoretical system that might execute operations on either CPU
or GPU. There are Scan and Join operators. For every operator, there are
two physical alternatives - execute on CPU or execute on GPU. We also have
a valid cost model that provides comparable costs for both CPU and GPU
backends. Both CPU and GPU could execute one operator at a time.

Now consider that we have the following logical plan:
LogicalJoin
  LogicalScan[a]
  LogicalScan[b]

We then expand the MEMO with physical alternatives (omitting some
alternatives for clarity):
Set#1 {
  Subset#1[CPU]: CpuJoin[Subset#3, Subset#5]
}
Set#2 {
  Subset#3[CPU]: {
CpuScan[a], cost=[100] (best)
GpuToCpu[Subset#4], cost=[170]
  }
  Subset#4[GPU]: {
GpuScan[a], cost=[150]
  }
}
Set#3 {
  Subset#5[CPU]: {
CpuScan[b], cost=[100] (best)
GpuToCpu[Subset#6], cost=[170]
  }
  Subset#6[GPU]: {
GpuScan[b], cost=[150]
  }
}

With the current model, Apache Calcite will only consider "best" rels from
each subset when constructing the final tree, which means that the
following tree would be the winner:
CpuJoin, cost=[200]
  CpuScan[a], cost=[100] (best from Subset#3)
  CpuScan[b], cost=[100] (best from Subset#5)

However, the better plan might be those, which utilizes the cross-device
parallelism:
CpuJoin, cost=MAX[100, 170]
  CpuScan[a], cost=[100] (best from Subset#3)
  GpuToCpu[b], cost=[170] (*not best* from Subset#5)
GpuScan[b], cost=[150]

It seems that to support this kind of optimization with non-additive costs,
we need to switch from cost calculation on the operator level (local) to a
cost calculation that considers alternative paths for all subtrees
(global). Obviously, this could lead to a combinatorial explosion easily.
On the other hand, the new top-down optimizer with a more predictable rule
invocation order might significantly amortize the additional complexity.

I want to ask whether similar ideas were discussed in the community before?
Are you aware of any practical systems that do such kind of optimization?
Or any fundamental results that prove this idea to be infeasible?
I would appreciate any hints on the matter.

Regards,
Vladimir.


Make RelNode attribute order part of MEMO

2021-03-07 Thread Vladimir Ozerov
Hi,

Currently, the order of attributes is used to define operators equivalence.
This leads to several interesting problems, such as possible duplication of
expressions (e.g., "a,a,b") or additional work in rules to detect trivial
projects and/or input permutations (e.g. "a,b -> b,a").

But the biggest problem is the join order planning. In Calcite, AxB is not
equivalent to BxA:

   1. It makes the ruleset [JoinCommuteRule, JoinAssociateRule]
   insufficient to explore all bushy trees because the commute operation adds
   a project on top of the new join (AxB -> project(BxA)), thus not
   allowing for the associate rule to be executed on the upper join. The
   solution is to add the ProjectJoinTransposeRule to the ruleset, but this
   increases the planning time dramatically, making Apache Calcite unsuitable
   for the cost-based join planning even for relatively simple join graphs.
   2. It increases the number of join-related rule calls, which complicates
   the implementation of the new join enumeration planning rules (e.g.
   top-down enumerators) because duplicate derivations compromise performance.

My question is - has the community considered an idea to make the order of
columns a common property of all operators, somewhat similar to the trait,
but without an explicit enforcer?

For example, consider the following MEMO which the planner creates when
trying to transform the join AxB to BxA:

G1: { Scan1[table=t1, cols=a,b] }
G2: { Scan2[table=t2, cols=c,d] }
G3: { AxB[G1, G2], Project[G4, cols=$2,$3,$0,$1] }
G4: { BxA[G2, G1] }

However, if we make the column order part of the MEMO, we may potentially
have something like this:

G1: { Scan1[table=t1, cols=a,b] }
G2: { Scan2[table=t2, cols=c,d] }
G3 [cols=G1.$0, G1.$1, G2.$0, G2.$1]: { AxB[G1, G2], BxA[G2, G1] }

Notice, how we were able to put AxB and BxA to the same equivalence group.
To my knowledge, CockroachDB uses a somewhat similar design.

I realize that this is rather a radical idea, and more design work is
required to come with a proper proposal. At this point, I just would like
to kindly ask the community to share high-level feedback on that. Were
similar ideas proposed before?

Thank you,
Vladimir.


Re: Question about parallel query planning

2021-03-09 Thread Vladimir Ozerov
Hi Jihoon,

I would say that 5 sec could be actually a pretty good result at such. Did
you have a chance to check which exact rules contributed to the planning
time? You may inject a listener to VolcanoPlanner to check that.

Regards,
Vladimir

Вт, 9 марта 2021 г. в 05:37, Jihoon Son :

> Hi all,
>
> I posted the same question on the ASF slack channel, but am posting
> here as well to get a quicker response.
>
> I'm seeing an issue in query planning that it takes a long time (+5
> sec) for a giant union query that has 120 subqueries in it. I captured
> a flame graph (attached in this email) to see where the bottleneck is,
> and based on the flame graph, I believe the query planner spent most
> of time to explore the search space of candidate plans to find the
> best plan. This seems because of those many subqueries in the same
> union query. Is my understanding correct? If so, for this particular
> case, it seems possible to parallelize exploring the search space. Do
> you have any plan for parallelizing this part? I'm not sure whether
> it's already done though in the master branch. I tried to search for a
> jira ticket on https://issues.apache.org/jira/browse/CALCITE, but
> couldn't find anything with my search skill.
>
> Thanks,
> Jihoon
>


Re: Question about parallel query planning

2021-03-09 Thread Vladimir Ozerov
*at such = at such scale

Вт, 9 марта 2021 г. в 11:01, Vladimir Ozerov :

> Hi Jihoon,
>
> I would say that 5 sec could be actually a pretty good result at such. Did
> you have a chance to check which exact rules contributed to the planning
> time? You may inject a listener to VolcanoPlanner to check that.
>
> Regards,
> Vladimir
>
> Вт, 9 марта 2021 г. в 05:37, Jihoon Son :
>
>> Hi all,
>>
>> I posted the same question on the ASF slack channel, but am posting
>> here as well to get a quicker response.
>>
>> I'm seeing an issue in query planning that it takes a long time (+5
>> sec) for a giant union query that has 120 subqueries in it. I captured
>> a flame graph (attached in this email) to see where the bottleneck is,
>> and based on the flame graph, I believe the query planner spent most
>> of time to explore the search space of candidate plans to find the
>> best plan. This seems because of those many subqueries in the same
>> union query. Is my understanding correct? If so, for this particular
>> case, it seems possible to parallelize exploring the search space. Do
>> you have any plan for parallelizing this part? I'm not sure whether
>> it's already done though in the master branch. I tried to search for a
>> jira ticket on https://issues.apache.org/jira/browse/CALCITE, but
>> couldn't find anything with my search skill.
>>
>> Thanks,
>> Jihoon
>>
>


Re: [DISCUSS] Default disable the RexNode normalization(or operands reorder)

2021-03-11 Thread Vladimir Ozerov
in our practice, we also had some problems with normalization. First, we
observed problems with the unwanted (and sometimes
incorrect) simplification of expressions with CASTs and literals which came
from RexSimplify. I couldn't find an easy way to disable that behavior.
Note, that RexSimplify may also be considered a "normalization". Second, we
implemented a way to avoid Project when doing join reordering but had some
issues with operator signatures due to lack of automatic normalization for
expressions for permuted inputs. These two cases demonstrate two opposite
views: sometimes you want a specific normalization to happen automatically,
but sometimes you want to disable it.

Perhaps an alternative approach could be to unify all simplification and
normalization logic and split it into configurable rules. Then, we may add
these rules as a separate rule set to the planner, which would be invoked
heuristically every time an operator with expressions is registered in
MEMO. In this case, a user would not need to bother about RexNode
constructors. To clarify, under "rules" I do not mean heavy-weight rules
similar to normal rules. Instead, it might be simple pattern+method pairs,
that could even be compiled into a static program using Janino. This
approach could be very flexible and convenient: a single place in the code
where all rewrite happens, complete control of the optimization rules,
modular rules instead of monolithic code (like in RexSimplify). The obvious
downside - it would require more time to implement than other proposed
approaches.

What do you think about that?

Regards,
Vladimir.

чт, 11 мар. 2021 г. в 13:33, Vladimir Sitnikov :

> Stamatis>just the option to use it or not in a more friendly way
> Stamatis>than a system property.
>
> As far as I remember, the key issue here is that new RexBuilder(...) is a
> quite common pattern,
> and what you suggest looks like "everyone would have to provide extra
> argument when creating RexBuilder".
>
> On top of that, there are use cases like "new RexCall(...)" in the static
> context (see org.apache.calcite.rex.RexUtil#not).
>
> Making the uses customizable adds significant overhead with doubtful gains.
>
> I have not explored the route though, so there might be solutions.
> For instance, it might work if we have an in-core dependency injection that
> would hide the complexity
> when coding :core, however, I don't think we could expose DI to Calcite
> users.
>
> Vladimir
>


Re: Make RelNode attribute order part of MEMO

2021-03-13 Thread Vladimir Ozerov
Hi Julian,

Thank you for sharing these issues. We end up with almost the same ideas.
We attempted to add an input permute to Join, which allowed us to avoid
projects. However, that complicates the integration with other rules, just
as you mention in [2]. Globally unique column IDs seem like a better option
from that perspective. Moreover, unique IDs may simplify the implementation
of other optimizations. For example, many join enumeration techniques,
whether DP-based or top-down, require the decomposition of the join graph
into independent vertices (inputs) and edges (conditions) and careful
reconstruction of the alternative join trees, and RexInputRef is not very
suitable for that process. Another possible example is recently reported
[3].

There are hundreds of usages of the RexInputRef, so the implementation of
this idea might be prohibitively expensive. But putting this problem aside,
do you envision any other potential blockers for the implementation of that
idea?

Regards,
Vladimir.

[1] https://github.com/apache/calcite/pull/2359
[2] https://issues.apache.org/jira/browse/CALCITE-62
[3] https://issues.apache.org/jira/browse/CALCITE-4534

вт, 9 мар. 2021 г. в 22:40, Julian Hyde :

> I investigated something similar a long time ago. We noticed that a
> lot of trivial Project operators were being generated to compensate
> for field re-ordering due to join transposition. And so the idea was
> to allow each RelNode (and especially Join) to permute its output
> fields.
>
> Here is the case: https://issues.apache.org/jira/browse/CALCITE-62.
> https://issues.apache.org/jira/browse/CALCITE-55 is related.
>
> The problem, as I noted in CALCITE-62, is that people writing rules
> have another mapping to deal with.
>
> I believe that other systems, such as Spark's Catalyst planner, use
> globally unique IDs for columns (as opposed to Calcite, whose column
> references are only locally unique, ordinals of the input
> operator(s)). Globally unique IDs would be superior for this problem
> but perhaps introduce other challenges.
>
> Julian
>
> On Sun, Mar 7, 2021 at 11:25 PM Vladimir Ozerov 
> wrote:
> >
> > Hi,
> >
> > Currently, the order of attributes is used to define operators
> equivalence.
> > This leads to several interesting problems, such as possible duplication
> of
> > expressions (e.g., "a,a,b") or additional work in rules to detect trivial
> > projects and/or input permutations (e.g. "a,b -> b,a").
> >
> > But the biggest problem is the join order planning. In Calcite, AxB is
> not
> > equivalent to BxA:
> >
> >1. It makes the ruleset [JoinCommuteRule, JoinAssociateRule]
> >insufficient to explore all bushy trees because the commute operation
> adds
> >a project on top of the new join (AxB -> project(BxA)), thus not
> >allowing for the associate rule to be executed on the upper join. The
> >solution is to add the ProjectJoinTransposeRule to the ruleset, but
> this
> >increases the planning time dramatically, making Apache Calcite
> unsuitable
> >for the cost-based join planning even for relatively simple join
> graphs.
> >2. It increases the number of join-related rule calls, which
> complicates
> >the implementation of the new join enumeration planning rules (e.g.
> >top-down enumerators) because duplicate derivations compromise
> performance.
> >
> > My question is - has the community considered an idea to make the order
> of
> > columns a common property of all operators, somewhat similar to the
> trait,
> > but without an explicit enforcer?
> >
> > For example, consider the following MEMO which the planner creates when
> > trying to transform the join AxB to BxA:
> >
> > G1: { Scan1[table=t1, cols=a,b] }
> > G2: { Scan2[table=t2, cols=c,d] }
> > G3: { AxB[G1, G2], Project[G4, cols=$2,$3,$0,$1] }
> > G4: { BxA[G2, G1] }
> >
> > However, if we make the column order part of the MEMO, we may potentially
> > have something like this:
> >
> > G1: { Scan1[table=t1, cols=a,b] }
> > G2: { Scan2[table=t2, cols=c,d] }
> > G3 [cols=G1.$0, G1.$1, G2.$0, G2.$1]: { AxB[G1, G2], BxA[G2, G1] }
> >
> > Notice, how we were able to put AxB and BxA to the same equivalence
> group.
> > To my knowledge, CockroachDB uses a somewhat similar design.
> >
> > I realize that this is rather a radical idea, and more design work is
> > required to come with a proper proposal. At this point, I just would like
> > to kindly ask the community to share high-level feedback on that. Were
> > similar ideas proposed before?
> >
> > Thank you,
> > Vladimir.
>


Re: [DISCUSS] Default disable the RexNode normalization(or operands reorder)

2021-03-13 Thread Vladimir Ozerov
Hi Julian,

I agree that in your example normalization may have some different concerns
comparing to simplification. However, both normalization and simplification
sometimes address similar problems either. For example, the simplification
may decrease the search space, but so does the normalization. E.g.
normalized reordering of operands in a join condition may allow for the
merge of equivalent nodes that otherwise would be considered
non-equivalent. Do any of the currently implemented rules depend on some
normalized representation?

Also, as many rules (such as join reorder rules) generate filters, I would
argue that moving the normalization to a separate phase might cause the
unnecessary expansion of the search space.

The idea I expressed above is inspired by CockroachDB (again :-)). In
CockroachDB, expressions are part of the MEMO and treated similarly to
relational operators, which allows for the unified rule infrastructure for
both operators and expressions. Expressions are created using a
context-aware builder, which knows the set of active normalization rules.
Whenever a builder is to create a new expression (not necessarily
the top-level), the normalization rules are invoked in a heuristic manner.
The code generation is used to build the heuristic rule executor. Both
normalization and simplification (in our terms) rules are invoked here. For
example, see [1] (normalization) and [2] (simplification). Finally, the
expression is registered in MEMO. As a result, every expression ever
produced is always in a normalized/simplified form.

I am not saying that we should follow this approach. But IMO (1) unified
handling of simplification and normalization through rules and (2) a single
entry point for all normalization (builder) are interesting design
decisions, as they offer both flexibility and convenience.

Regards,
Vladimir.

[1]
https://github.com/cockroachdb/cockroach/blob/release-21.1/pkg/sql/opt/norm/rules/scalar.opt#L8
[2]
https://github.com/cockroachdb/cockroach/blob/release-21.1/pkg/sql/opt/norm/rules/bool.opt#L30

пт, 12 мар. 2021 г. в 07:15, Julian Hyde :

> Without simplifications, many trivial RelNodes would be produced. It is
> beneficial to have those in RelBuilder; if they were in rules, the trivial
> RelNodes (and equivalence sets) would still be present, increasing the size
> of the search space.
>
> I want to draw a distinction between simplification and normalization. A
> normalized form is relied upon throughout the system. Suppose for example,
> that we always normalize ‘RexLiteral = RexInputRef’ to ‘RexInputRef =
> RexLiteral’. If a rule encountered the latter case, it would not be a bug
> if the rule failed with, say, a ClassCastException.
>
> So, I disagree with Vladimir that 'RexSimplify may also be considered a
> “normalization”’. If simplification is turned off, each rule must be able
> to deal with the unsimplified expressions.
>
> Also, the very idea of normalizations being optional, enabled by system
> properties or other config, is rather disturbing, because the rules
> probably don’t know that the normalization has been turned off.
>
> The only place for normalization, in my opinion, is explicitly, in a
> particular planner phase. For example, pulling up all filters before
> attempting to match materialized views.
>
> Julian
>
> > On Mar 11, 2021, at 10:37 AM, Vladimir Ozerov 
> wrote:
> >
> > in our practice, we also had some problems with normalization. First, we
> > observed problems with the unwanted (and sometimes
> > incorrect) simplification of expressions with CASTs and literals which
> came
> > from RexSimplify. I couldn't find an easy way to disable that behavior.
> > Note, that RexSimplify may also be considered a "normalization". Second,
> we
> > implemented a way to avoid Project when doing join reordering but had
> some
> > issues with operator signatures due to lack of automatic normalization
> for
> > expressions for permuted inputs. These two cases demonstrate two opposite
> > views: sometimes you want a specific normalization to happen
> automatically,
> > but sometimes you want to disable it.
> >
> > Perhaps an alternative approach could be to unify all simplification and
> > normalization logic and split it into configurable rules. Then, we may
> add
> > these rules as a separate rule set to the planner, which would be invoked
> > heuristically every time an operator with expressions is registered in
> > MEMO. In this case, a user would not need to bother about RexNode
> > constructors. To clarify, under "rules" I do not mean heavy-weight rules
> > similar to normal rules. Instead, it might be simple pattern+method
> pairs,
> > that could even be compiled into a static program using Janino. This

Trait propagation in heterogeneous plans

2021-03-27 Thread Vladimir Ozerov
Hi,

Apache Calcite supports heterogeneous optimization when nodes may have
different conventions. The Enumerable rules propagate all traits from
inputs. We have doubts whether this is correct or not.

Consider the following initial plan, which was created by Apache Calcite
after sql-to-rel conversion and invocation of TranslatableTable.toRel. The
table is in the CUSTOM convention. In this convention, there is an
additional Distribution trait that tracks which attribute is used for
sharding. It could be either SHARDED or ANY. The latter is the default
distribution value which is used when the distribution is unknown. Suppose
that the table is distributed by the attribute $0.
LogicalProject [convention=NONE,   distribution=ANY]
  CustomTable  [convention=CUSTOM, distribution=SHARDED($0)]

Now suppose that we run VolcanoPlanner with two rules: EnumerableProjectRule
and converter rules that translate the CUSTOM node to ENUMERABLE node.
First, the EnumerableProjectRule is executed. This rule propagates traits
from the input, replacing only convention. Notice, how it propagated the
distribution trait.
EnumerableProject [convention=ENUMERABLE, distribution=SHARDED($0)]
  CustomTable [convention=CUSTOM, distribution=SHARDED($0)]

Next, the converter will be invoked, yielding the following final plan:
EnumerableProject[convention=ENUMERABLE, distribution=SHARDED($0)]
  CustomToEnumerable [convention=ENUMERABLE, distribution=???]
CustomTable  [convention=CUSTOM, distribution=SHARDED($0)]

There are two problems here. First, the project operator potentially
destroys any trait which depends on column order, such as distribution or
collation. Therefore, EnumerableProject has an incorrect value of the
distribution trait.
Second, which distribution should I assign to the CustomToEnumerable node?
As I know that parent convention cannot handle the distribution properly,
my natural thought is to set it to ANY. However, at least in the top-down
optimizer, this will lead to CannotPlanException, unless I declare that [ANY
satisfies SHARDED($0)], which is not the case: ANY is unknown distribution,
so all distribution satisfies ANY, but not vice versa.

My question is - shouldn't we ensure that only the collation trait is
propagated from child nodes in Enumerable rules? For example, in the
EnumerableProjectRule instead of doing:
input.getTraitSet()
  .replace(EnumerableConvention.INSTANCE)
  .replace()

we may do:
RelOptCluster.traitSet().
  .replace(EnumerableConvention.INSTANCE)
  .replace()

This would ensure that all other traits are set to the default value. The
generalization of this idea is that every convention has a set of supported
traits. Every unsupported trait should be set to the default value.

I would appreciate your feedback on the matter.

Regards,
Vladimir.


Time zone management and DST in Avatica

2021-03-28 Thread Vladimir Ozerov
Hi,

Avatica assumes that the underlying engine operates on relative time
without time zone as opposed to global time (UTC). When you set a temporal
value (e.g., a prepared statement parameter), Avatica adds the current
offset to the passed time. When you read a temporal value, Avatica
subtracts the current offset. This may lead to incorrect results if DST
offset changes.

Consider that we have a timezone with DST, that works as follows. D1 and D2
are two consecutive days (e.g., 24 and 25 Oct):
D2 00:00 GMT+2 -> D1 22:00 GMT
D2 01:00 GMT+2 -> D1 23:00 GMT
D2 03:00 GMT+3 -> D2 00:00 GMT
D2 04:00 GMT+3 -> D2 01:00 GMT

Now consider, that we want to save D2 00:00 GMT+2 using Avatica. On write,
Avatica will advance the time by the TZ offset. On read, Avatica will
subtract the TZ offset. The problem is that different offsets will be used,
leading to the incorrect result. The associated logic is located in
AbstractCursor and TypedValue classes.

long initial = [D2 00:00 GMT+2].epochMillis() // D1 22:00 GMT
long onWrite = initial + offsetAt(initial);   // D2 00:00 GMT
long onRead = onWrite - offsetAt(onWrite);// D1 21:00 GMT
assert initial == onRead; // Fails

The fundamental problem is that the current time offset is used, which
might differ before and after adjustment. One potential solution is to
enhance the reading part. It should check whether the offset after the
subtraction is the same and if not - do the additional adjustment to
restore the proper time.

Do you have any objections to the proposed change?

Regards,
Vladimir.


Re: Time zone management and DST in Avatica

2021-03-31 Thread Vladimir Ozerov
Hi,

Thank you for your feedback. Let me provide the exact example of when the
problem happened. Consider the date 2020 Oct 25 in Israel Standard Time
[1]. There was DST end at this date. This date represents exactly 18560
days since 01 Jan 1970. When there was 2020 Oct 25 00:00 IST, it was 2020
Oct 24 21:00 GMT. Therefore, my expectation is that if I return 18560 to
Avatica, I would get  2020 Oct 24 21:00 GMT millis back. This is what at
least PG and SQL Server returns. You may check it in the SQLFiddle [1]
quickly:

Schema:
create table t(c int primary key);
insert into t values (1);

Postgres:
select
  cast('October 25 00:00:00 2020 Israel' as timestamp with time zone) at
time zone 'UTC'
from t;

SQL Server:
select
  ({d'2020-10-25'}  AT TIME ZONE 'Israel Standard Time') AT TIME ZONE 'UTC'
from t;

However, Avatica returns me 2020 Oct 24 22:00 GMT. This is wrong because at
22:00 GMT it was 01:00 on a wall clock in Israel.
The problem is that we cannot use the current time to derive the offset
that should be applied to the past time.

Regards,
Vladimir.

[1] https://www.timeanddate.com/time/zone/israel/jerusalem
[2] http://sqlfiddle.com/

ср, 31 мар. 2021 г. в 10:38, Vladimir Sitnikov :

> Julian>you should be able to come up with a test case
> Julian>against a reputable database (e.g. PostgreSQL
>
> pgjdbc committer here :)
>
> There are timezone-related tests in
>
> https://github.com/pgjdbc/pgjdbc/blob/c633cc6cf5295bfbd5b6a79bb45dff863c5056f5/pgjdbc/src/test/java/org/postgresql/test/jdbc2/TimezoneTest.java
>
> and
>
> https://github.com/pgjdbc/pgjdbc/blob/c633cc6cf5295bfbd5b6a79bb45dff863c5056f5/pgjdbc/src/test/java/org/postgresql/test/jdbc42/GetObject310Test.java
>
> It might be fun to run the tests through Avatica.
>
> Vladimir, frankly speaking, I don't follow what do you mean by "Avatica
> assumes that the underlying engine operates on relative time"
> Could you please provide the exact API call sequence and the expected
> result?
>
> Vladimir
>


Re: Time zone management and DST in Avatica

2021-03-31 Thread Vladimir Ozerov
Hi Julian, Vladimir,

Israel time zone It is not fixed. Formally, you may find different
abbreviations, like IST and IDT. But in real systems, like Postgres, MS
SQL, or Java, this is a single time zone with DST changes.

The following query in Postgres will return 21:00, 21:00, 22:00 for 24, 25,
and 26 Oct respectively:
select
  cast('October 24 00:00:00 2020 Israel' as timestamp with time zone) at
time zone 'UTC',
  cast('October 25 00:00:00 2020 Israel' as timestamp with time zone) at
time zone 'UTC',
  cast('October 26 00:00:00 2020 Israel' as timestamp with time zone) at
time zone 'UTC'
from t;

Same for SQL Server, even though the zone is named "Israel *Standard* Time"

In Java, the relevant ZoneId is "Israel". And it also returns 21:00, 21:00,
22:00 for these dates:

ZoneId utcZoneId = ZoneId.of("UTC");
ZoneId istZoneId = ZoneId.of("Israel");
LocalDate date24 = LocalDate.of(2020, 10, 24);
LocalDate date25 = LocalDate.of(2020, 10, 25);
LocalDate date26 = LocalDate.of(2020, 10, 26);
System.out.println(ZonedDateTime.ofInstant(date24.atStartOfDay(istZoneId).toInstant(),
utcZoneId));
System.out.println(ZonedDateTime.ofInstant(date25.atStartOfDay(istZoneId).toInstant(),
utcZoneId));
System.out.println(ZonedDateTime.ofInstant(date26.atStartOfDay(istZoneId).toInstant(),
utcZoneId));

I do not have an isolated test for Avatica for now, but I observed the
problem in the real system. The problematic code is in
AbstractCursor.longToDate. The method accepts the absolute GMT time at the
start of the day. E.g., 25-Oct-2020 00:00 GMT. At this time Israel's offset
was +2:00. Then it subtracts that offset, assuming that  25-Oct-2020 00:00
IST == 24-Oct-2020 22:00 GMT. But it is not, because several hours earlier
the offset was +3:00 due to DST. So actually 25-Oct-2020 00:00 IST ==
24-Oct-2020 21:00 GMT. As a result, Avatica will return 22:00 from the
result set, while other databases and Java would return 21:00.

I can write a dedicated test for that, but it would require some time. But
I do not see much value in it during the discussion, because the problem is
relatively clear: the offset at time T2 cannot be used to deduce the offset
at time T1.

This is why other drivers often do some "magic" with Calendar to get the
correct time. Like in PG JDBC [1]. Notice, that for time zones without DST,
they just do some simple math, similarly to Avatica. But for time zones
with DST, they do more complicated calculations to get the correct result.

Regards,
Vladimir.

[1]
https://github.com/pgjdbc/pgjdbc/blob/866c6a9e4cc42d9c279d68b8c756f562eaf0f249/pgjdbc/src/main/java/org/postgresql/jdbc/TimestampUtils.java#L1329

ср, 31 мар. 2021 г. в 23:25, Julian Hyde :

> Israel Standard Time is fixed at UTC+2. It does not observe daylight
> savings time. So maybe your wall clock should have been in the
> 'Asia/Jerusalem' time zone rather than Israel Standard Time.
>
> On Wed, Mar 31, 2021 at 12:23 PM Vladimir Sitnikov
>  wrote:
> >
> > >Let me provide the exact example
> >
> > Sorry for not being clear.
> > As far as I understand, Avatica means "Java JDBC API" or "something like
> Go
> > API".
> >
> > Could you please provide a test case via Avatica API (e.g. JDBC API)
> along
> > with the actual and expected result?
> > For example, org.apache.calcite.jdbc.CalciteRemoteDriverTest verifies the
> > behavior of Avatica+Calcite
> > integration.
> >
> > Vladimir
>


Re: Trait propagation in heterogeneous plans

2021-05-04 Thread Vladimir Ozerov
Hi Haisheng,

My original problem was with how Enumerable propagates traits. Many
Enumerable rules copy traits from the child operator. This seems wrong
because, as you mentioned, Enumerable supports only collation. Propagation
of the unsupported traits may lead to CannotPlanException as in the example
above when having a plan with multiple conventions.

Therefore, the proposal is to change Enumerable rules, so that they
propagate only collation, but not other traits. Does it make sense?

Regards,
Vladimir.

ср, 21 апр. 2021 г. в 04:31, Haisheng Yuan :

> Hi Vladimir,
>
> > There are two problems here. First, the project operator potentially
> > destroys any trait which depends on column order, such as distribution or
> > collation. Therefore, EnumerableProject has an incorrect value of the
> > distribution trait.
>
> The enumerable convention is intended for in-memory, non-distributed
> environment.
> Therefore, we only consider 2 traits: collation and convention. Other
> traits are not
> guaranteed to work correctly. If you want it work with distribution, you
> have to create
> your own operators, rules, either by extending or overriding, in which
> case, you will need
> to remap distribution columns to get the correct distribution trait, just
> like how collation does.
>
> > Second, which distribution should I assign to the CustomToEnumerable
> node?
> > As I know that parent convention cannot handle the distribution properly,
> > my natural thought is to set it to ANY.
>
> You can assume CustomToEnumerable to be an Enforcer operator, like Sort,
> Exchange.
> Sort only changes data collation, Exchange changes data distribution and
> collation, similarly
> CustomToEnumerable only change convention, but retains collation and
> distribution, I assume.
> But in practice, it should be decided by the operator inventor and the
> underlying physical
> implementation.
>
> Hope that answers your question. Feel free to ask if you have more
> questions.
>
> Thanks,
> Haisheng Yuan
>
> On 2021/03/27 08:43:15, Vladimir Ozerov  wrote:
> > Hi,
> >
> > Apache Calcite supports heterogeneous optimization when nodes may have
> > different conventions. The Enumerable rules propagate all traits from
> > inputs. We have doubts whether this is correct or not.
> >
> > Consider the following initial plan, which was created by Apache Calcite
> > after sql-to-rel conversion and invocation of TranslatableTable.toRel.
> The
> > table is in the CUSTOM convention. In this convention, there is an
> > additional Distribution trait that tracks which attribute is used for
> > sharding. It could be either SHARDED or ANY. The latter is the default
> > distribution value which is used when the distribution is unknown.
> Suppose
> > that the table is distributed by the attribute $0.
> > LogicalProject [convention=NONE,   distribution=ANY]
> >   CustomTable  [convention=CUSTOM, distribution=SHARDED($0)]
> >
> > Now suppose that we run VolcanoPlanner with two rules:
> EnumerableProjectRule
> > and converter rules that translate the CUSTOM node to ENUMERABLE node.
> > First, the EnumerableProjectRule is executed. This rule propagates traits
> > from the input, replacing only convention. Notice, how it propagated the
> > distribution trait.
> > EnumerableProject [convention=ENUMERABLE, distribution=SHARDED($0)]
> >   CustomTable [convention=CUSTOM, distribution=SHARDED($0)]
> >
> > Next, the converter will be invoked, yielding the following final plan:
> > EnumerableProject[convention=ENUMERABLE, distribution=SHARDED($0)]
> >   CustomToEnumerable [convention=ENUMERABLE, distribution=???]
> > CustomTable  [convention=CUSTOM, distribution=SHARDED($0)]
> >
> > There are two problems here. First, the project operator potentially
> > destroys any trait which depends on column order, such as distribution or
> > collation. Therefore, EnumerableProject has an incorrect value of the
> > distribution trait.
> > Second, which distribution should I assign to the CustomToEnumerable
> node?
> > As I know that parent convention cannot handle the distribution properly,
> > my natural thought is to set it to ANY. However, at least in the top-down
> > optimizer, this will lead to CannotPlanException, unless I declare that
> [ANY
> > satisfies SHARDED($0)], which is not the case: ANY is unknown
> distribution,
> > so all distribution satisfies ANY, but not vice versa.
> >
> > My question is - shouldn't we ensure that only the collation trait is
> > propagated from child nodes in Enumerable rules? For example, in the
> >

Re: Trait propagation in heterogeneous plans

2021-05-04 Thread Vladimir Ozerov
Hi Vladimir,

I couldn't share the reproducer, as it is behind the NDA. But the problem
is evident from the code.

There are two distinct issues actually:

   1. Propagation of unsupported traits in operators. EnumerableProject is
   not affected. Examples of the problem: EnumerableWindowRule,
   EnumerableFilterRule
   2. Incorrect enforcement of the input traits. Example:
   EnumerableProjectRule.convert. Imagine that I have an input with some
   custom trait, say, distribution. The EnumerableProjectRule may require
   the input to satisfy some specific distribution. But given that the
   distribution is not supported by Enumerable, I want to destroy the
   distribution in my convention enforcer. If I do so, I get the
   CannotPlanException, because the created EnumerableProject incorrectly
   requires the specific distribution from the input.

Regards,
Vladimir.

вт, 4 мая 2021 г. в 11:06, Vladimir Sitnikov :

> >First, the EnumerableProjectRule is executed. This rule propagates traits
> >from the input, replacing only convention.
>
> Vladimir, could you please share a reproducer?
>
> EnumerableProject#create explicitly resets all the traits for
> EnumerableProject except convention=enumerable, and
> collation=computed_with_metadataquery
> In practice, it could compute distribution traits as well, however, that is
> missing.
>
> Are you sure you get EnumerableProject with non-default distribution
> somehow?
>
> Vladimir
>


Re: Trait propagation in heterogeneous plans

2021-05-05 Thread Vladimir Ozerov
Hi Vladimir, Julian,

I want to distinguish between two cases.

Some projects may decide to use Calcite's distribution trait. To my
knowledge, this is not a common pattern because it is not really integrated
into Calcite. It is not destroyed/adjusted in rules and operators as
needed, not integrated into EnumerableConvention.enforce, etc.

Other projects may decide to use a custom distribution trait. Examples are
Apache Flink, Hazelcast, and some other private projects we work on. There
are many reasons to do this. A couple of examples:
1. Calcite's distribution produces logical exchange, while production
grade-optimizers are typically multi-phase and want the distribution
convention to produce physical exchanges in a dedicated physical phase(s).
2. Some systems may have custom requirements for distribution, such as
propagating the number of shards, supporting multiple equivalent keys, etc.

But in both cases, the bottom line is that the Enumerable currently cannot
work with both built-in and custom distributions because the associated
code is not implemented in Calcite's core. And even if we add the
fully-fledged support of the built-in distribution to Enumerable, many
projects will continue using custom distribution traits because the
exchange is a physical operation with lots of backend-dependent specific
quirks, and any attempt to model it abstractly in Calcite's core is
unlikely to cover some edge cases.

The same applies to any other custom trait that depends on columns -
Enumerable will not be able to process it correctly.

Therefore, instead of having a definitively broken code, it might be better
to apply the defensive approach when the whole Enumerable backend provides
a clear and consistent contract: we support collation and reset everything
else. IMO it is better because it matches the current behavior and would
never cause strange bugs in a user code. If in the future we invest in the
proper integration of the built-in distribution or figure out how to
"externalize" the trait propagation for Enumerable operators, we may relax
this statement.

Please let me know if it makes any sense.

Regards,
Vladimir.

вт, 4 мая 2021 г. в 21:02, Julian Hyde :

> > I would say known in-core vs unknown trait is a reasonable approach to
> > distingush traits.
>
> Easy, but not reasonable. It will make it very difficult to reuse
> existing rels and rules (e.g. Enumerable) in a downstream project that
> has defined its own traits.
>
> On Tue, May 4, 2021 at 10:44 AM Vladimir Sitnikov
>  wrote:
> >
> > > It seems arbitrary to include Collation but exclude other traits.
> >
> > I would say known in-core vs unknown trait is a reasonable approach to
> > distingush traits.
> >
> > Vladimir
>


Re: Trait propagation in heterogeneous plans

2021-05-06 Thread Vladimir Ozerov
Hi,

I'd like to stress out that I am not trying to argue about subjective
concepts at all. Quite the opposite - I would like to agree or disagree on
a set of objective facts and find the solution. Specifically, from what I
saw in Calcite's codebase and real projects, I assert the following:

   1. Calcite-based projects may use custom traits.
   2. Enumerable in its current state cannot propagate any traits except
   for collation. The relevant code is simply missing from the product, it was
   never implemented.
   3. Despite (2), Enumerable rules/operators may demand unsupported traits
   from inputs, or expose unsupported traits, which may lead to problems on
   the user side (an example is in the first message of this thread).

Do you agree with these points?

If we are in agreement here, then I propose only one thing - fix (3),
because it affects real-life integrations. The fix is trivial:

   - Make sure that Enumerable operators never set non-default trait values
   for anything except for collation. For example, EnumerableProjectRule
   creates an operator with the correct trait set, whilst
   EnumerableFilterRule propagates unsupported traits.
   - Replace RelNode.getTraitSet with RelOptCluster.traitSet when deducing
   the desired input trait set in Enumerable rules.

These two fixes would ensure that we never have any non-default values of
any traits except for collation in Enumerable operators. On the one hand,
it fixes (3). On the other hand, it doesn't break anything, because thanks
to (2) there is nothing to break.

Does it make sense to you?

Regards,
Vladimir.


чт, 6 мая 2021 г. в 10:35, Vladimir Sitnikov :

> Vladimir,
>
> I generally agree with what you are saying,
>
> >Enumerable backend provides a clear and consistent contract: we support
> collation and reset everything
>
> That sounds like a way to go until there's a way to externalize "input
> trait enforcement" rules.
> "output" traits are simpler since they can be computed with metadataquery
> (however, we still hard-code the set of computed traits).
> It might be worth trying to compute all the traits known to the planner.
>
> However, Enumerable could play well with in-core distribution trait as
> well, so there's no need to limit enumerable to "collation only".
>
> If you don't like in-core distribution trait, you just do not use it.
> There's no much sense in limiting enumerable to collation only.
>
> Vladimir
>


Re: Trait propagation in heterogeneous plans

2021-05-06 Thread Vladimir Ozerov
It may propagate the in-core distribution in theory, if the relevant code
exists. Practically, there is no such code. For example, consider
EnumerableProject:

   1. EnumerableProjectRule.convert doesn't propagate input's distribution,
   thanks to EnumerableProject.create that uses RelOptCluster.traitSet.
   2. EnumerableProjectRule.derive also ignores all traits except for
   collation.

Therefore, irrespective of which trait set is present in the project's
input, the EnumerableProject will always have the default values for all
traits except for collation. This is what I refer to as "no trait
propagation". In this sense, EnumerableProject is an example of the correct
implementation wrt my proposal. But not all operators follow this, e.g.
EnumerableFilter.

чт, 6 мая 2021 г. в 14:39, Vladimir Sitnikov :

> >Enumerable in its current state cannot propagate any traits except for
> collation
>
> Enumerable can propagate in-core distribution trait.
>
> Vladimir
>


AggregateUnionTransposeRule fails when some inputs have unique grouping key

2021-05-19 Thread Vladimir Ozerov
Hi,

The AggregateUnionTransposeRule attempts to push the Aggregate below the
Union.

Before:
Aggregate[group=$0, agg=SUM($1]
  Union[all]
Input1
Input2

After:
Aggregate[group=$0, agg=SUM($1]
  Union[all]
Aggregate[group=$0, agg=SUM($1]
  Input1
Aggregate[group=$0, agg=SUM($1]
  Input2

When pushing the Aggregate, it checks whether the input is definitively
unique on the grouping key. If yes, the Aggregate is not installed on top
of the input, assuming that the result would be the same as without the
Aggregate. This generates a type mismatch exception when aggregation is
pushed only to some of the inputs:
Aggregate[group=$0, agg=SUM($1]
  Union[all]
Aggregate[group=$0, agg=SUM($1]
  Input1
Input2

It seems that the uniqueness check should not be in that rule at all, and
the aggregate should be pushed unconditionally. Motivation: we already have
AggregateRemoveRule that removes unnecessary aggregates. No need to
duplicate the same non-trivial logic twice.

Does the proposal make sense to you?

Regards,
Vladimir.


Re: AggregateUnionTransposeRule fails when some inputs have unique grouping key

2021-05-21 Thread Vladimir Ozerov
Hi,

I created two issues for two distinct bugs in AggregateUnionTransposeRule:

   1. https://issues.apache.org/jira/browse/CALCITE-4616 - the problem with
   a partial Aggregate push-down
   2. https://issues.apache.org/jira/browse/CALCITE-4615 - the problem with
   an incorrect type of aggregate call (not discussed in this thread before)

Regarding issue 1, my proposal is to remove the partial pushdown
optimization completely. We may do the following:

   1. Iterate over all inputs, and check if they have a unique key on the
   Aggregate's group key.
   2. If all inputs are unique, return as there is no sense to push the
   Aggregate past Union.
   3. If at least one input is not unique, push aggregate to *all* inputs,
   to maintain the proper row type, assuming that the AggregateRemoveRule
   will eventually remove unnecessary aggregations.

Regards,
Vladimir.

пт, 21 мая 2021 г. в 09:43, JiaTao Tao :

> Sounds  make sense
>
> Regards!
>
> Aron Tao
>
>
> Vladimir Ozerov  于2021年5月19日周三 下午8:57写道:
>
> > Hi,
> >
> > The AggregateUnionTransposeRule attempts to push the Aggregate below the
> > Union.
> >
> > Before:
> > Aggregate[group=$0, agg=SUM($1]
> >   Union[all]
> > Input1
> > Input2
> >
> > After:
> > Aggregate[group=$0, agg=SUM($1]
> >   Union[all]
> > Aggregate[group=$0, agg=SUM($1]
> >   Input1
> > Aggregate[group=$0, agg=SUM($1]
> >   Input2
> >
> > When pushing the Aggregate, it checks whether the input is definitively
> > unique on the grouping key. If yes, the Aggregate is not installed on top
> > of the input, assuming that the result would be the same as without the
> > Aggregate. This generates a type mismatch exception when aggregation is
> > pushed only to some of the inputs:
> > Aggregate[group=$0, agg=SUM($1]
> >   Union[all]
> > Aggregate[group=$0, agg=SUM($1]
> >   Input1
> > Input2
> >
> > It seems that the uniqueness check should not be in that rule at all, and
> > the aggregate should be pushed unconditionally. Motivation: we already
> have
> > AggregateRemoveRule that removes unnecessary aggregates. No need to
> > duplicate the same non-trivial logic twice.
> >
> > Does the proposal make sense to you?
> >
> > Regards,
> > Vladimir.
> >
>


Exposing multiple values of a trait from the operator

2021-05-25 Thread Vladimir Ozerov
Hi,

Consider the distributed SQL engine that uses a distribution property to
model exchanges. Consider the following physical tree. To do the
distributed join, we co-locate tuples using the equijoin key. Now the Join
operator has two equivalent distributions - [a1] and [b1]. It is critical
to expose both distributions so that the top Aggregate can take advantage
of the co-location.

Aggregate[group=b1]
  DistributedJoin[a.a1=b.b1]   // SHARDED[a1], SHARDED[b1]
Input[a]   // SHARDED[a1]
Input[b]   // SHARDED[b1]

A similar example for the Project:
Aggregate[group=$1]
  Project[$0=a, $1=a] // SHARDED[$0], SHARDED[$1]
Input // SHARDED[a]

The question is how to model this situation properly?

First, it seems that RelMultipleTrait and RelCompositeTrait were designed
to handle this situation. However, I couldn't make them work with the
top-down optimizer. The reason is that when we register a RelNode with a
composite trait in MEMO, VolcanoPlanner flattens the composite trait into
the default trait value in RelSet.add -> RelTraitSet.simplify. That is, the
trait [SHARDED[a], SHARDED[b]] will be converted to [ANY] so that the
original traits could not be derived in the PhysicalNode.derive methods.

Second, we may try to model multiple sharding keys in a single trait. But
this complicates the implementation of PhysicalNode.passThrough/derive
significantly.
SHARDED[a1, a2], SHARDED[b1, b2] -> SHARDED[[a1, a2], [b1, b2]]

Third, we may expose multiple traits using metadata. RelMdDistribution
would not work, because it exposes only a single distribution. But a custom
handler may potentially fix that. However, it will not be integrated with
the top-down optimizer still, which makes the idea questionable.

To summarize, it seems that currently there is no easy way to handle
composite traits with a top-down optimizer. I wonder whether someone from
the devlist already solved similar issues in Apache Calcite or other
optimizers. If so, what was the approach or best practices? Intuitively, it
seems that RelMultipleTrait/RelCompositeTrait approach might be the way to
go. But why do we replace the original composite trait set with the default
value in the RelTraitSet.simplify routine?

Regards,
Vladimir.


Re: Exposing multiple values of a trait from the operator

2021-05-25 Thread Vladimir Ozerov
Hi Vladimir,

Thank you for the link. It is very relevant to my problem. I see that in
these discussions, there were several ideas and claims, such as that (1) we
can get rid of "simplify" altogether, (2) composite traits are rare in
practice, (3) composite traits are not designed well in the first place
[1]. I do not have the full picture in my head, so I'll try to share some
thoughts to advance the discussion.

Regarding (1), I think that the removal of "simplify" may help with my
particular (and pretty simple) test but might lead to some unpredictable
results for more complicated queries. Suppose that we generate two
equivalent nodes with different traits: [a] and [a][b]. Depending on the
nature of the trait def, these two nodes might or might belong to the same
subset. For example, [a] and [a][b] are different subsets for
RelCollcation. At the same time, [a] and [a][b] could belong to the same
subset for some distributions. That is, if the input is hash-distributed by
either [a] or [b], it might imply that a==b for every tuple (otherwise,
hashes will not match), and therefore every RelNode in the RelSet that is
shared by [a] is also sharded by [b] and vice verse. The idea is similar to
transitive predicates. So ideally, we should let the RelTraitDef define how
to compare composite traits with other traits. Otherwise, we may lose some
optimization opportunities.

Regarding (2), perhaps the multi-collation nodes are really rare in
practice. But nodes with multiple hash distributions are widespread for
distributed engines. Because in distributed systems, the collocated hash
equijoin is the most common way of joining two inputs, and such join always
produces an additional distribution.

Regarding (3), it would be very interesting to hear suggestions and ideas
on the proper design of composite traits. The composite traits mechanics
mentioned in RelSubset Javadoc's is not a good design choice for
distribution traits. That is, if we have a node that is distributed by
[a][b], we cannot just put it into two subsets [a] and [b], because
operator parents may require both [a] and [b], otherwise unnecessary
exchanges could appear. That is, [a][b] should be propagated together. For
example, the removal of SHARDED[a1] from #1 would add the exchange between
#2 and #1, and the removal of SHARDED[b1] from #1 would add the exchange
between #3 and #2. Neither is optimal.
3: Aggregate[group=b1]
2:   Join[a.a1=c.c1]   // SHARDED[a1], SHARDED[b1], SHARDED[c1]
1: Join[a.a1=b.b1] // SHARDED[a1], SHARDED[b1]

@Haisheng Yuan , following your comment [1], would
you mind providing your ideas around the proper design of composite traits?
Are composite traits implemented in Orca?

Regards,
Vladimir.

[1]
https://issues.apache.org/jira/browse/CALCITE-2593?focusedCommentId=17081984&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17081984

вт, 25 мая 2021 г. в 19:32, Vladimir Sitnikov :

> >VolcanoPlanner flattens the composite trait into
> the default trait value in RelSet.add -> RelTraitSet.simplify
>
> Vladimir, have you tried removing that RelTraitSet.simplify?
>
> I remember I have run into that multiple times already, and I suggested
> removing that "simplify".
> For example,
>
> https://issues.apache.org/jira/browse/CALCITE-2593?focusedCommentId=16750377&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16750377
>
>
> Vladimir
>


Re: Exposing multiple values of a trait from the operator

2021-05-25 Thread Vladimir Ozerov
Hi Haisheng,

Thank you for the advice. This is exactly how I designed distribution at
the moment (the approach 2 from my original email) - as a List
instead of just int[]. My main concern was the increased complexity of the
trait propagation/derivation, as I have to manage these nested lists by
hand. Nevertheless, it works well. So I hoped that there are better
built-in approaches that I may use. If the answer is negative, I'll
continue using the original approach, when multiple alternatives managed
manually.

Regards,
Vladimir.

вт, 25 мая 2021 г. в 20:30, Haisheng Yuan :

> Hi Vladimir,
>
> Glad to see you raised the question.
>
> Here is the advice:
> Do not use RelMultipleTrait/RelCompositeTrait, which is fundamentally
> flawed and has many bugs. It can't work properly no matter for top-down or
> bottom-up.
>
> Instead, we need to add equivalent keys bitmap as the property of physical
> trait like RelCollation, RelDistribution.
>
> For example:
> class RelDistributionImpl {
>   // list of distribution keys
>   private ImmutableIntList keys;
>
>// list of equivalent bitset for each distribution key
>   private ImmutableList equivBitSets;
> }
>
> In the trait satisfy and column remapping, we also need to take equivalent
> keys into consideration. Some of the work need to be done in Calcite core
> framework.
>
> Greenplum Orca optimizer has similar strategy:
>
> https://github.com/greenplum-db/gporca/blob/master/libgpopt/include/gpopt/base/CDistributionSpecHashed.h#L44
>
> Regards,
> Haisheng Yuan
>
> On 2021/05/25 15:37:32, Vladimir Ozerov  wrote:
> > Hi,
> >
> > Consider the distributed SQL engine that uses a distribution property to
> > model exchanges. Consider the following physical tree. To do the
> > distributed join, we co-locate tuples using the equijoin key. Now the
> Join
> > operator has two equivalent distributions - [a1] and [b1]. It is critical
> > to expose both distributions so that the top Aggregate can take advantage
> > of the co-location.
> >
> > Aggregate[group=b1]
> >   DistributedJoin[a.a1=b.b1]   // SHARDED[a1], SHARDED[b1]
> > Input[a]   // SHARDED[a1]
> > Input[b]   // SHARDED[b1]
> >
> > A similar example for the Project:
> > Aggregate[group=$1]
> >   Project[$0=a, $1=a] // SHARDED[$0], SHARDED[$1]
> > Input // SHARDED[a]
> >
> > The question is how to model this situation properly?
> >
> > First, it seems that RelMultipleTrait and RelCompositeTrait were designed
> > to handle this situation. However, I couldn't make them work with the
> > top-down optimizer. The reason is that when we register a RelNode with a
> > composite trait in MEMO, VolcanoPlanner flattens the composite trait into
> > the default trait value in RelSet.add -> RelTraitSet.simplify. That is,
> the
> > trait [SHARDED[a], SHARDED[b]] will be converted to [ANY] so that the
> > original traits could not be derived in the PhysicalNode.derive methods.
> >
> > Second, we may try to model multiple sharding keys in a single trait. But
> > this complicates the implementation of PhysicalNode.passThrough/derive
> > significantly.
> > SHARDED[a1, a2], SHARDED[b1, b2] -> SHARDED[[a1, a2], [b1, b2]]
> >
> > Third, we may expose multiple traits using metadata. RelMdDistribution
> > would not work, because it exposes only a single distribution. But a
> custom
> > handler may potentially fix that. However, it will not be integrated with
> > the top-down optimizer still, which makes the idea questionable.
> >
> > To summarize, it seems that currently there is no easy way to handle
> > composite traits with a top-down optimizer. I wonder whether someone from
> > the devlist already solved similar issues in Apache Calcite or other
> > optimizers. If so, what was the approach or best practices? Intuitively,
> it
> > seems that RelMultipleTrait/RelCompositeTrait approach might be the way
> to
> > go. But why do we replace the original composite trait set with the
> default
> > value in the RelTraitSet.simplify routine?
> >
> > Regards,
> > Vladimir.
> >
>


Re: AggregateUnionTransposeRule fails when some inputs have unique grouping key

2021-05-25 Thread Vladimir Ozerov
Done:
https://issues.apache.org/jira/browse/CALCITE-4616?focusedCommentId=17351269&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17351269

пт, 21 мая 2021 г. в 21:32, Julian Hyde :

> Can you add your proposed fix to the bug, and we can discuss there.
>
> > On May 21, 2021, at 2:33 AM, Vladimir Ozerov  wrote:
> >
> > Hi,
> >
> > I created two issues for two distinct bugs in
> AggregateUnionTransposeRule:
> >
> >   1. https://issues.apache.org/jira/browse/CALCITE-4616 - the problem
> with
> >   a partial Aggregate push-down
> >   2. https://issues.apache.org/jira/browse/CALCITE-4615 - the problem
> with
> >   an incorrect type of aggregate call (not discussed in this thread
> before)
> >
> > Regarding issue 1, my proposal is to remove the partial pushdown
> > optimization completely. We may do the following:
> >
> >   1. Iterate over all inputs, and check if they have a unique key on the
> >   Aggregate's group key.
> >   2. If all inputs are unique, return as there is no sense to push the
> >   Aggregate past Union.
> >   3. If at least one input is not unique, push aggregate to *all* inputs,
> >   to maintain the proper row type, assuming that the AggregateRemoveRule
> >   will eventually remove unnecessary aggregations.
> >
> > Regards,
> > Vladimir.
> >
> > пт, 21 мая 2021 г. в 09:43, JiaTao Tao :
> >
> >> Sounds  make sense
> >>
> >> Regards!
> >>
> >> Aron Tao
> >>
> >>
> >> Vladimir Ozerov  于2021年5月19日周三 下午8:57写道:
> >>
> >>> Hi,
> >>>
> >>> The AggregateUnionTransposeRule attempts to push the Aggregate below
> the
> >>> Union.
> >>>
> >>> Before:
> >>> Aggregate[group=$0, agg=SUM($1]
> >>>  Union[all]
> >>>Input1
> >>>Input2
> >>>
> >>> After:
> >>> Aggregate[group=$0, agg=SUM($1]
> >>>  Union[all]
> >>>Aggregate[group=$0, agg=SUM($1]
> >>>  Input1
> >>>Aggregate[group=$0, agg=SUM($1]
> >>>  Input2
> >>>
> >>> When pushing the Aggregate, it checks whether the input is definitively
> >>> unique on the grouping key. If yes, the Aggregate is not installed on
> top
> >>> of the input, assuming that the result would be the same as without the
> >>> Aggregate. This generates a type mismatch exception when aggregation is
> >>> pushed only to some of the inputs:
> >>> Aggregate[group=$0, agg=SUM($1]
> >>>  Union[all]
> >>>Aggregate[group=$0, agg=SUM($1]
> >>>  Input1
> >>>Input2
> >>>
> >>> It seems that the uniqueness check should not be in that rule at all,
> and
> >>> the aggregate should be pushed unconditionally. Motivation: we already
> >> have
> >>> AggregateRemoveRule that removes unnecessary aggregates. No need to
> >>> duplicate the same non-trivial logic twice.
> >>>
> >>> Does the proposal make sense to you?
> >>>
> >>> Regards,
> >>> Vladimir.
> >>>
> >>
>
>


Trait propagation guidelines

2021-05-26 Thread Vladimir Ozerov
Hi,

I tried to optimize a certain combination of operators for the distributed
engine and got stuck with the trait propagation in the top-down engine. I
want to ask the community for advice on whether the problem is solvable
with the current Apache Calcite implementation or not.

Consider the following logical tree:
3: LogicalAggregate[group=[a], F2(c)]
2:  LogicalAggregate[group=[a,b], F1(c)]
1:LogicalScan[t]

Consider that these two aggregates cannot be merged or simplified for
whatever reason. We have only a set of physical rules to translate this
logical tree to a physical tree. Also, there could be any number of
other operators between these two aggregates. We omit them for clarity,
assuming that the distribution is not destroyed.

In the distributed environment, non-collocated aggregates are often
implemented in two phases: local pre-aggregation and final aggregation,
with an exchange in between. Consider that the Scan operator is hash
distributed by some key other than [a] or [b]. If we optimize operators
without considering the whole plan, we may optimize each operator
independently, which would give us the following plan:
3: PhysicalAggregate[group=[a], F2_phase2(c)] //
HASH_DISTRIBUTED [a]
3:   Exchange[a]  //
HASH_DISTRIBUTED [a]
3: PhysicalAggregate[group=[a], F2_phase1(c)] //
HASH_DISTRIBUTED [a,b]
2:   PhysicalAggregate[group=[a,b], F1_phase2(c)] //
HASH_DISTRIBUTED [a,b]
2: Exchange[a, b] //
HASH_DISTRIBUTED [a,b]
2:   PhysicalAggregate[group=[a,b], F1_phase1(c)] //
HASH_DISTRIBUTED [d]
1: PhysicalScan[t]//
HASH_DISTRIBUTED [d]

This plan is not optimal, because we re-hash inputs twice. A better plan
that we want to get:
3: PhysicalAggregate[group=[a], F2(c)]// HASH_DISTRIBUTED
[a]
2:   PhysicalAggregate[group=[a,b], F1_phase2(c)] // HASH_DISTRIBUTED
[a]
2: Exchange[a]// HASH_DISTRIBUTED
[a]
2:   PhysicalAggregate[group=[a,b], F1_phase1(c)] // HASH_DISTRIBUTED
[d]
1: PhysicalScan[t]// HASH_DISTRIBUTED
[d]

In this case, we take advantage of the fact that the distribution [a] is
compatible with [a,b]. Therefore we may enforce only [a], instead of doing
[a,b] and then [a]. Since exchange operators are very expensive, this
optimization may bring a significant boost to the query engine. Now the
question - how do we reach that state? Intuitively, a pass-through is
exactly what we need. We may pass the optimization request from top
aggregate to bottom aggregate to find physical implementations shared by
[a]. But the devil is in the details - when and how exactly to pass this
request?

Typically, we have a conversion rule that converts a logical aggregate to a
physical aggregate. We may invoke "convert" on the input to initiate the
pass-through:

RelNode convert(...) {
return new PhysicalAggregate(
convert(input, HASH_DISTRIBUTED[a])
)
}

The first problem - we cannot create the normal physical aggregate here
because we do not know input traits yet. The final decision whether to do a
one-phase or two-phase aggregate can be made only in the
"PhysicalNode.derive" method when concrete input traits are resolved.
Therefore the converter rule should create a kind of "template" physical
operator, which would be used to construct the final operator(s) when input
traits are resolved. AFAIU Enumerable works similarly: we create operators
with virtually arbitrary traits taken from logical nodes in the conversion
rules. We only later do create normal nodes in the derive() methods.

The second problem - our top aggregate doesn't actually need the
HASH_DISTRIBUTED[a] input. Instead, it may accept inputs with any
distribution. What we really need is to inform the input (bottom aggregate)
that it should look for additional implementations that satisfy
HASH_DISTRIBUTED[a]. Therefore, enforcing a specific distribution on the
input using the "convert" method is not what we need because this
conversion might enforce unnecessary exchanges.

The third problem - derivation. Consider that we delivered the optimization
request to the bottom aggregate. As an implementor, what am I supposed to
do in this method? I cannot return the final aggregate from here because
the real input traits are not derived yet. Therefore, I can only return
another template, hoping that the "derive" method will be called on it.
However, this will not happen because trait derivation is skipped on the
nodes emitted from pass-through. See "DeriveTrait.perform" [1].

BottomAggregate {
RelNode passThrough(distribution=HASH_DISTRIBUTED[a]) {
// ???
}
}

I feel that I am either going in the wrong direction, or some gaps in the
product disallow such optimization. So I would like to ask the community to
assist with the following questions:
1. In the top-down optim

Re: Trait propagation guidelines

2021-05-27 Thread Vladimir Ozerov
> 2. Given a desired traitset from parent node, the current relnode only
> needs to generate a single relnode after passing down the traitset. Given a
> traitset delivered by child node, the current relnode only derive a single
> relnode. Quite unlike other optimizer, in Calcite's top-down optimizer, you
> don't need to worry about issuing multiple optimization requests to inputs,
> which is handled by Calcite framework secretly. i.e.
> > SELECT a, b, min(c) from foo group by a, b;
> > In many other optimizer, we probably need ask the aggregate to issue 3
> distribution requests for tablescan on foo, which are
> > 1) hash distributed by a,
> > 2) hash distributed by b,
> > 3) hash distributed by a, b
> > However in Calcite top-down optimizer, your physical implementation rule
> for global aggregate only need generate a single physical node with hash
> distribution by a, b. In case the table foo happens to be distributed by a,
> or b, the derive() method will tell you there is an opportunity. This is
> the feature that Calcite's top-down optimizer excels over other optimizers,
> because this can dramatically reduce the search space while keeping the
> optimal optimization opportunity.
> >
> > 3. This is by design. Nodes produced from "passThrough" and "derive" and
> just sibling physical node with different traitset, we only need the
> initial physical nodes after implementation to avoid unnecessary
> operations. The fundamental reason is, unlike Orca optimizer where physical
> node and physical property are separate things, Calcite's logical/physical
> nodes contains traitset. With regard to the latter question, can you give
> an example?
> >
> > Regards,
> > Haisheng Yuan
> >
> >
> > On 2021/05/26 20:11:57, Vladimir Ozerov  wrote:
> > > Hi,
> > >
> > > I tried to optimize a certain combination of operators for the
> distributed
> > > engine and got stuck with the trait propagation in the top-down
> engine. I
> > > want to ask the community for advice on whether the problem is solvable
> > > with the current Apache Calcite implementation or not.
> > >
> > > Consider the following logical tree:
> > > 3: LogicalAggregate[group=[a], F2(c)]
> > > 2:  LogicalAggregate[group=[a,b], F1(c)]
> > > 1:LogicalScan[t]
> > >
> > > Consider that these two aggregates cannot be merged or simplified for
> > > whatever reason. We have only a set of physical rules to translate this
> > > logical tree to a physical tree. Also, there could be any number of
> > > other operators between these two aggregates. We omit them for clarity,
> > > assuming that the distribution is not destroyed.
> > >
> > > In the distributed environment, non-collocated aggregates are often
> > > implemented in two phases: local pre-aggregation and final aggregation,
> > > with an exchange in between. Consider that the Scan operator is hash
> > > distributed by some key other than [a] or [b]. If we optimize operators
> > > without considering the whole plan, we may optimize each operator
> > > independently, which would give us the following plan:
> > > 3: PhysicalAggregate[group=[a], F2_phase2(c)] //
> > > HASH_DISTRIBUTED [a]
> > > 3:   Exchange[a]  //
> > > HASH_DISTRIBUTED [a]
> > > 3: PhysicalAggregate[group=[a], F2_phase1(c)] //
> > > HASH_DISTRIBUTED [a,b]
> > > 2:   PhysicalAggregate[group=[a,b], F1_phase2(c)] //
> > > HASH_DISTRIBUTED [a,b]
> > > 2: Exchange[a, b] //
> > > HASH_DISTRIBUTED [a,b]
> > > 2:   PhysicalAggregate[group=[a,b], F1_phase1(c)] //
> > > HASH_DISTRIBUTED [d]
> > > 1: PhysicalScan[t]//
> > > HASH_DISTRIBUTED [d]
> > >
> > > This plan is not optimal, because we re-hash inputs twice. A better
> plan
> > > that we want to get:
> > > 3: PhysicalAggregate[group=[a], F2(c)]//
> HASH_DISTRIBUTED
> > > [a]
> > > 2:   PhysicalAggregate[group=[a,b], F1_phase2(c)] //
> HASH_DISTRIBUTED
> > > [a]
> > > 2: Exchange[a]//
> HASH_DISTRIBUTED
> > > [a]
> > > 2:   PhysicalAggregate[group=[a,b], F1_phase1(c)] //
> HASH_DISTRIBUTED
> > > [d]
> > > 1: PhysicalScan[t]//
> HASH_DISTRIBUTED
> > > [d]
> > >
> > > In this ca

Re: Trait propagation guidelines

2021-05-27 Thread Vladimir Ozerov
; ...
>
> // Candidate 6. passThrough called on candidate2
> PhysicalAggregate[group=[a], F2_phase2(c)]
>   PhysicalAggregate[group=[a,b], F1_phase2(c)]  // deliver dist[a]
> Exchange[dist[a]]
>   PhysicalAggregate[group=[a,b], F1_phase1(c)]
> ...
>
> step 3:
> The cost model chooses the best candidate.
> Note that Candidate 5 is not always the best. For example, when it is
> detected, from stats or other, that data is skewed on key [a], Candidate 2
> may be better. When it is detected that NDV(a, b) = 0.99 * ROWCOUNT() ,
> Candidate 6 is preferred, as partial aggregate can reduce little data. So
> it is not wasty to build all those candidates.
>
> Most of the above works are done by calcite frameworks. Users only need to:
> 1. Fire both implementations during aggregation builds.
> 2. Overwrite the passThroughTraits method.
>
> Thanks,
> Jinpeng Wu
>
>
> On Thu, May 27, 2021 at 8:19 AM Haisheng Yuan  wrote:
>
> > Another point I would like to mention is that it is not recommended to
> > override method "passThrough" and "derive" directly, override
> > "passThroughTraits" and "deriveTraits" instead, so that we can make sure
> > only the same type of physical node is created and no nested relnodes or
> > additional RelSets are created, unless you know you have to create
> > different type of nodes. For example, if the table foo has an btree index
> > on column a, and the parent relnode is requesting ordering on column a,
> > then we may consider to override "passThrough" of TableScan to return an
> > IndexScan instead of a TableScan.
> >
> > Regards,
> > Haisheng Yuan
> > On 2021/05/26 22:45:20, Haisheng Yuan  wrote:
> > > Hi Vladimir,
> > >
> > > 1. You need a logical rule to split the aggregate into a local
> aggregate
> > and global aggregate, for example:
> > >
> >
> https://github.com/greenplum-db/gporca/blob/master/libgpopt/src/xforms/CXformSplitGbAgg.cpp
> > > Only implementation rules can convert a logical node to a physical node
> > or multiple physical nodes.
> > > After physical implementation, you have 2 physical alternatives:
> > > 1) single phase global physical aggregate,
> > > 2) 2 phase physical aggregate with local and global aggregate.
> > > It should be up to the cost to decide which one to choose.
> > >
> > > 2. Given a desired traitset from parent node, the current relnode only
> > needs to generate a single relnode after passing down the traitset.
> Given a
> > traitset delivered by child node, the current relnode only derive a
> single
> > relnode. Quite unlike other optimizer, in Calcite's top-down optimizer,
> you
> > don't need to worry about issuing multiple optimization requests to
> inputs,
> > which is handled by Calcite framework secretly. i.e.
> > > SELECT a, b, min(c) from foo group by a, b;
> > > In many other optimizer, we probably need ask the aggregate to issue 3
> > distribution requests for tablescan on foo, which are
> > > 1) hash distributed by a,
> > > 2) hash distributed by b,
> > > 3) hash distributed by a, b
> > > However in Calcite top-down optimizer, your physical implementation
> rule
> > for global aggregate only need generate a single physical node with hash
> > distribution by a, b. In case the table foo happens to be distributed by
> a,
> > or b, the derive() method will tell you there is an opportunity. This is
> > the feature that Calcite's top-down optimizer excels over other
> optimizers,
> > because this can dramatically reduce the search space while keeping the
> > optimal optimization opportunity.
> > >
> > > 3. This is by design. Nodes produced from "passThrough" and "derive"
> and
> > just sibling physical node with different traitset, we only need the
> > initial physical nodes after implementation to avoid unnecessary
> > operations. The fundamental reason is, unlike Orca optimizer where
> physical
> > node and physical property are separate things, Calcite's
> logical/physical
> > nodes contains traitset. With regard to the latter question, can you give
> > an example?
> > >
> > > Regards,
> > > Haisheng Yuan
> > >
> > >
> > > On 2021/05/26 20:11:57, Vladimir Ozerov  wrote:
> > > > Hi,
> > > >
> > > > I tried to optimize a certain combination of operators for the
> > distributed
> > > > engine and got stuck with the trait propagation in the top-

Re: Trait propagation guidelines

2021-05-28 Thread Vladimir Ozerov
ty requirement among child operators. Unlike Orca optimizer, where
> there is exact match, partial satisfying, orderless match etc, Calcite's
> default implementation always require exact satisfying. But we can still
> make use of "passThrough" and "derive" to achieve our goal. i.e. the
> aggregate generated by implementation rule requires itself and its child to
> delivered distribution on [a,b], but the "derive" method tells Aggregate
> that [b,a] is available, it can generate another option to require [b,a]
> instead.
> >
> > > In distributed engines, the nodes emitted from rules are basically
> "templates"
> > > that must be replaced with normal nodes.
> >
> > There is no difference between distributed and non-distributed engines
> when dealing with this. In Orca and CockroachDB optimizer, the nodes
> emitted from rules are operators without physical properties, the optimizer
> then request physical properties in top-down manner, either recursively or
> stack, or state machine. Calcite is quite different. when the physical
> operator is generated by implementation rule, the physical operator must
> has its own traits, at the same time, the traits that it expects its child
> operators to deliver. So in Calcite, they are not "templates". The
> difference is there since Calcite's inception.
> >
> > Regards,
> > Haisheng Yuan
> >
> > On 2021/05/27 08:59:33, Vladimir Ozerov  wrote:
> > > Hi Haisheng,
> > >
> > > Thank you for your inputs. They are really helpful. Let me summarize
> your
> > > feedback in my own words to verify that I understand it correctly.
> > >
> > >1. In distributed systems, an implementation rule may produce
> different
> > >physical operators depending on the input traits. Examples are
> Aggregate,
> > >Sort, Window.
> > >2. Since input traits are not known when the rule is fired, we must
> > >generate *all possible combinations* of physical operators that we
> may
> > >need. For LogicalAggregate, we must generate 1-phase and 2-phase
> > >alternatives. For LogicalSort, we also have 1-phase and 2-phase
> > >alternatives. Etc.
> > >3. If all combinations are generated, it is expected that
> "passThrough"
> > >and "derive" would be just trivial replacements of traits for most
> cases.
> > >This is why "passThroughTraits" and "deriveTraits" are recommended.
> A
> > >notable exception is TableScan that may emit alternative indexes in
> > >response to the pass-through requests.
> > >
> > > If my understanding is correct, then there are several issues with this
> > > approach still.
> > >
> > > 1. Unlike Aggregate or Sort, which may have only 1 or 2 phases, certain
> > > logical operators may have many physical alternatives. Consider the
> Window
> > > function:
> > > SELECT
> > >   AGG1 over (partition by a),
> > >   AGG2 over (partition by b),
> > >   AGG3 over (partition by c),
> > >   ...
> > > FROM input
> > >
> > > To calculate each aggregate, we need to re-shuffle the input based on
> the
> > > partition key. The key question is the order of reshuffling. If the
> input
> > > is shared by [a], I want to calculate AGG1 locally and then re-shuffle
> the
> > > input to calculate other aggregates. For the remaining AGG2 and AGG3,
> the
> > > order is also important. If the parent demands sharding by [b], then
> the
> > > proper sequence is b-c-a:
> > > 1: Window[AGG2 over (partition by b)] // SHARDED[b]
> > > 2:   Window[AGG3 over (partition by c)]   // SHARDED[c]
> > > 3: Window[AGG1 over (partition by a)] // SHARDED[a]
> > > 4:   Input// SHARDED[a]
> > >
> > > But if the parent demands [c], the proper sequence is c-b-a. Since we
> do
> > > not know real distributions when the rule is fired, we must emit all
> the
> > > permutations to ensure that no optimization opportunity is missed. But
> with
> > > complex window aggregate, this might be impractical because we will
> emit
> > > lots of unnecessary nodes.
> > >
> > > 2. As input traits are not known when the rule is fired, the nodes
> emitted
> > > from the implementation rules most likely would not be used in the
> final
> > > plan. For example, I can create a physical aggregate that demands
> > > non-strict distribu

Top-down optimizer cannot explore the search space because physical rule is treated as transformation rule

2021-05-28 Thread Vladimir Ozerov
Hi,

I have an optimizer that uses top-down VolcanoPlanner and has a
ConverterRule for every LogicalNode. I have a new requirement when one of
the physical rules must emit several physical nodes instead of one. I tried
to convert a ConverterRule to a normal rule that emits physical nodes. Even
though the new rule has exactly the same pattern and logic as the previous
ConverterRule, plans changed. Analysis showed that this likely due to a bug
in the top-down optimizer as explained below.

When optimizing a logical node, the top-down first schedules the
transformation rules, and then implementation rules. The logic to check
whether the rule is transformation rule or not is located in
VolcanoPlanner.isTransformationRule [1]. The rule scheduling logic ensures
that a given rule is executed either as a transformation rule, or an
implementation rule, but not both. See TopDowRuleQueue.popMatch. The
top-down optimizer schedules tasks in a stack. So even though the
transformation rules are scheduled before implementation rules, the latter
executed first.

If an implementation rule produces a physical node, this node will be
notified about input traits in the "derive" phase. In contrast,
transformation rules produce logical nodes only, and this happens after the
derivation of the inputs is completed. Therefore, if the top-down optimizer
mistakenly treats an implementation rule as a transformation rule, "derive"
will not be called on the produced physical nodes, leading to incomplete
search space exploration.

It seems, that this is exactly what happens in the current implementation.
The VolcanoPlanner.isTransformationRule looks like this:

 1: protected boolean isTransformationRule(VolcanoRuleCall match) {
 2:if (match.getRule() instanceof SubstitutionRule) {
 3:  return true;
 4:}
 5:if (match.getRule() instanceof ConverterRule
 6:&& match.getRule().getOutTrait() == rootConvention) {
 7:  return false;
 8:}
 9:return match.getRule().getOperand().trait == Convention.NONE
10:|| match.getRule().getOperand().trait == null;
11: }

If the rule is a ConverterRule and it produces the node with the target
convention, it is treated as an implementation rule (lines 5-6). But if the
rule is not a ConverterRule, the method tries to deduce the rule's type
from the incoming convention (lines 9-10). In practice, implementation
rules either do not care about the incoming trait or expect the NONE trait.
Therefore, it seems that currently, the top-down optimizer treats many
implementation rules as physical rules, and as a result, cannot notify
physical nodes produced from these rules about trait derivation.

This explains why in my case everything was ok when all implementation
rules were ConverterRules, and why I lost some optimal plans when the rule
was refactored to a non-converter variant.

Do you agree that this a bug? If yes, shouldn't we refactor that code to
just check whether the rule is an instance of TransformationRule? Since
this is a breaking change, we may add a special flag that preserves the old
behavior by default but allows for the new behavior to overcome the
aforementioned problem.

Regards,
Vladimir.

[1]
https://github.com/apache/calcite/blob/calcite-1.26.0/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java#L1398-L1408


Re: Top-down optimizer cannot explore the search space because physical rule is treated as transformation rule

2021-05-28 Thread Vladimir Ozerov
Hi Jinpeng,

Thank you for the clarification. When I saw the code in question for the
first time, my first thought was that it was perhaps designed for gradual
migration. The main problem is that the current implementation discards
parts of the plan *silently*, which might be difficult to spot. I
only spotted the problem in my specific case because I had ~100 tests with
complex queries. Otherwise, I would happily proceed with the new rule
without knowing that I lost important parts of the search space.

That said, I think we can do the following:

   1. Emit a warning if or even throw an exception if the transformation
   rule produced a physical node. This should be trivial to implement - add an
   expected node type to VolcanoRuleCall (e.g., "logical", "physical", "any").
   The warning/exception should contain a proper fix suggestion - to override
   the VolcanoPlanner.isTransformationRule.
   2. Alternatively - do a breaking change. Apache Calcite doesn't have a
   major release cadence. It is normal practice in many products to do
   breaking changes in minor releases. Even popular products like Mongo or
   DataStax do it regularly. We may inform the user in the first release and
   change to "rule instanceof TransformationRule" in the next release.

Does it make sense?

Regards,
Vladimir.

пт, 28 мая 2021 г. в 19:33, Jinpeng Wu :

> Hi, Vladimir. Good catch! There could be some improvements here.
>
> Actually, this problem was discovered early when the top-down rule driver
> was designed. At that time, no rule was annotated as "TransformationRule".
> Moreover, it is impossible to ask every calcite user who designed their own
> rules to annotate the existing code. So the top-down rule driver was
> designed so that it can:
> 1. Work in chaos: even if there are no hints for rule types, it can still
> work. Some opportunities may be lost, but NO failures, NO exceptions, and
> NO worse than the original driver. People can migrate to the new driver
> without concern.
> 2. Be Improvable: Users can refactor their code, if they want, step by
> step. As rule types become more and more accurate, the system achieves more
> and more benefits
> 3. Be easy to customize: the default implementation is designed for the
> most common cases, so that most users can benefit from it without much
> effort. But it is not possible to fulfill all requirements as different
> systems could have very different patterns to define logical and
> physical. That's why the isTransformationRule method is put in
> VolcanoPlanner and marked as protected: overwriting it can be very simple.
>
> Moreover, losing some "derive" opportunities is not as serious as
> imagination. As I mentioned in previous discussions, parents are in charge
> of raising as many requirements as possible. During "derive", if specific
> traits were not built by children, it means that no parents were requiring
> that. And if parents finally require that traits in the latter
> optimization, passThrough methods get called and new physical nodes are
> generated and "derive" get called again.
> I tested it on millions of queries, with or without correct rule types, in
> my own product. The performance of group pruning varies a lot. But the
> output plans are almost the same. Only one obvious exception was
> discovered: the spool node. That's because spool nodes cannot "passThough"
> parent traits (it could have multiple parents and current framework cannot
> handle such a situation) while it can "derive" input traits.
>
> Of course, this conclusion may not apply to your product as we could have
> quite different rule sets. I am just sharing some of my experiences. Maybe
> the current implementation of "isTransformationRule" is not good enough. If
> you have any better solutions, please share them.
>
> Thanks,
> Jinpeng Wu
>
> On Fri, May 28, 2021 at 7:10 PM Vladimir Ozerov 
> wrote:
>
> > Hi,
> >
> > I have an optimizer that uses top-down VolcanoPlanner and has a
> > ConverterRule for every LogicalNode. I have a new requirement when one of
> > the physical rules must emit several physical nodes instead of one. I
> tried
> > to convert a ConverterRule to a normal rule that emits physical nodes.
> Even
> > though the new rule has exactly the same pattern and logic as the
> previous
> > ConverterRule, plans changed. Analysis showed that this likely due to a
> bug
> > in the top-down optimizer as explained below.
> >
> > When optimizing a logical node, the top-down first schedules the
> > transformation rules, and then implementation rules. The logic to check
> > whether the rule is transformation rule or not is 

Re: Top-down optimizer cannot explore the search space because physical rule is treated as transformation rule

2021-05-28 Thread Vladimir Ozerov
BTW, I tried to change the implementation to:

 1: protected boolean isTransformationRule(VolcanoRuleCall match) {
 2:return match.getRule() instanceof TransformationRule;
 3: }

It solved my problem - plans returned to normal. In the Apache Calcite
repo, only 4 tests in the TopDowOptTest class failed due to a minor
operator reordering.

пт, 28 мая 2021 г. в 20:37, Vladimir Ozerov :

> Hi Jinpeng,
>
> Thank you for the clarification. When I saw the code in question for the
> first time, my first thought was that it was perhaps designed for gradual
> migration. The main problem is that the current implementation discards
> parts of the plan *silently*, which might be difficult to spot. I
> only spotted the problem in my specific case because I had ~100 tests with
> complex queries. Otherwise, I would happily proceed with the new rule
> without knowing that I lost important parts of the search space.
>
> That said, I think we can do the following:
>
>1. Emit a warning if or even throw an exception if the transformation
>rule produced a physical node. This should be trivial to implement - add an
>expected node type to VolcanoRuleCall (e.g., "logical", "physical", "any").
>The warning/exception should contain a proper fix suggestion - to override
>the VolcanoPlanner.isTransformationRule.
>2. Alternatively - do a breaking change. Apache Calcite doesn't have a
>major release cadence. It is normal practice in many products to do
>breaking changes in minor releases. Even popular products like Mongo or
>DataStax do it regularly. We may inform the user in the first release and
>change to "rule instanceof TransformationRule" in the next release.
>
> Does it make sense?
>
> Regards,
> Vladimir.
>
> пт, 28 мая 2021 г. в 19:33, Jinpeng Wu :
>
>> Hi, Vladimir. Good catch! There could be some improvements here.
>>
>> Actually, this problem was discovered early when the top-down rule driver
>> was designed. At that time, no rule was annotated as "TransformationRule".
>> Moreover, it is impossible to ask every calcite user who designed their
>> own
>> rules to annotate the existing code. So the top-down rule driver was
>> designed so that it can:
>> 1. Work in chaos: even if there are no hints for rule types, it can still
>> work. Some opportunities may be lost, but NO failures, NO exceptions, and
>> NO worse than the original driver. People can migrate to the new driver
>> without concern.
>> 2. Be Improvable: Users can refactor their code, if they want, step by
>> step. As rule types become more and more accurate, the system achieves
>> more
>> and more benefits
>> 3. Be easy to customize: the default implementation is designed for the
>> most common cases, so that most users can benefit from it without much
>> effort. But it is not possible to fulfill all requirements as different
>> systems could have very different patterns to define logical and
>> physical. That's why the isTransformationRule method is put in
>> VolcanoPlanner and marked as protected: overwriting it can be very simple.
>>
>> Moreover, losing some "derive" opportunities is not as serious as
>> imagination. As I mentioned in previous discussions, parents are in charge
>> of raising as many requirements as possible. During "derive", if specific
>> traits were not built by children, it means that no parents were requiring
>> that. And if parents finally require that traits in the latter
>> optimization, passThrough methods get called and new physical nodes are
>> generated and "derive" get called again.
>> I tested it on millions of queries, with or without correct rule types, in
>> my own product. The performance of group pruning varies a lot. But the
>> output plans are almost the same. Only one obvious exception was
>> discovered: the spool node. That's because spool nodes cannot "passThough"
>> parent traits (it could have multiple parents and current framework cannot
>> handle such a situation) while it can "derive" input traits.
>>
>> Of course, this conclusion may not apply to your product as we could have
>> quite different rule sets. I am just sharing some of my experiences. Maybe
>> the current implementation of "isTransformationRule" is not good enough.
>> If
>> you have any better solutions, please share them.
>>
>> Thanks,
>> Jinpeng Wu
>>
>> On Fri, May 28, 2021 at 7:10 PM Vladimir Ozerov 
>> wrote:
>>
>> > Hi,
>> >
>> > I have an optimizer that uses top-down VolcanoPlan

Re: Trait propagation guidelines

2021-05-29 Thread Vladimir Ozerov
   +--  Broadcast
> > >+-- TableScan on bar
> > >
> > > Can you tell that the single phase aggregate plan is always better than
> > the 2 phase aggregate plan?
> > >
> > > > Therefore, the typical way to optimize
> > > > LogicalAggregate is to split in the physical phase (implementation
> > rule,
> > > > pass-through, derive). Practical systems like Dremio [1] and Flink
> [2]
> > > > work this way.
> > > Dremio and Flink work this way doesn't mean it is a good way. Greenplum
> > Orca and Alibaba MaxCompute optimizer work in another way. In Flink and
> > Dremio, they have HashAggPRule to generate 1 phase HashAgg and 2 phase
> > HashAgg, SortAggPRule to generate 1 phase SortAgg and 2 phase SortAgg.
> > However do you think there is possibility that the global SortAgg
> combined
> > with local HashAgg, or the global HashAgg combined with local SortAgg may
> > perform better in difference cases? Are you going to generate all the 4
> > combinations in the implementation rule? There are some cases we found
> we'd
> > better to split the aggregate into 3 phase aggregate [1], in which case,
> > will the implementation rule generate 3 HashAggs or 3 SortAggs, or all
> the
> > 6 combinations?
> > >
> > > In our system, we have 1 phase, 2 phase, 3 phase logical aggregate
> rules
> > to transform the LogicalAggregate to another kind of logical aggregate(s)
> > with phase info, say LogicalXXXAggregate, then our physical aggregate
> rules
> > match this kind of node to generate HashAgg or StreamAgg. Of course, in
> the
> > logical rules, we can add business logic to guess the possible traits
> > delivered by child nodes to determine whether the rule definitely won't
> > generate a better alternative and may decide to abort this transformation
> > early. But I would rather let the cost model decide.
> > >
> > > Admittedly, the current top-down optimization is not pure on-demand
> > request oriented, because it will always generate a physical request
> > regardless the parent nodes' trait request. For example the following
> query
> > in a non-distributed environment:
> > > select a, b, c, max(d) from foo group by a, b, c order by a desc;
> > >
> > > It will first generate a StreamAgg[a ASC, b ASC, c ASC] no matter what
> > the parent node requires, then the "passThrough" tells StreamAgg that
> > parent requires [a DESC], we get a StreamAgg[a DESC, b ASC, c ASC]. It
> > would be ideal if we only generate StreamAgg[a DESC, b ASC, c ASC] by
> > request, but I don't think that will make much difference, the bottleneck
> > relies on the join order enumeration and the Project related operation.
> > >
> > > Regards,
> > > Haisheng Yuan
> > >
> > > [1]
> >
> https://github.com/greenplum-db/gporca/blob/master/libgpopt/src/xforms/CXformSplitDQA.cpp
> > >
> > > On 2021/05/28 09:17:45, Vladimir Ozerov  wrote:
> > > > Hi Jinpeng, Haisheng,
> > > >
> > > > Thank you for your inputs. I really appreciate that. Let me try to
> > address
> > > > some of your comments and share some experience with the
> > implementation of
> > > > optimizers for a distributed engine I am currently working with.
> > > >
> > > > First of all, I would argue that multiple logical operators do not
> > have a
> > > > 1-1 mapping to physical operators, and Window is not special here.
> For
> > > > instance, LogicalAggregate doesn't have 1-1 mapping to physical
> > aggregates
> > > > because the physical implementation can be either 1-phase or 2-phase.
> > It
> > > > doesn't matter that the 2-phase aggregate is a composition of two
> > 1-phase
> > > > aggregates: the whole decision of whether to go with 1-phase or
> 2-phase
> > > > aggregate is a physical decision that should be made based on
> > available (or
> > > > assumed) input traits.
> > > >
> > > > Consider the following logical tree:
> > > > LogicalAggregate[group=$0, agg=SUM($1)]
> > > >   Input
> > > >
> > > > If I do the split on the logical phase with a separate transformation
> > rule,
> > > > I will get the following tree:
> > > > LogicalAggregate[group=$0, agg=SUM($1)]
> > > >   LogicalAggregate[group=$0, agg=SUM($1)]
> > > > Input
>

Re: Top-down optimizer cannot explore the search space because physical rule is treated as transformation rule

2021-05-29 Thread Vladimir Ozerov
Great. Does the community have any objections to the following fix?
1. Add a flag to a rule call instance with the expected node type. In the
case of a mismatch, we will print a warning once per rule per JVM instance
to avoid too many messages. Alternatively, we may print a warning per rule
per VolcanoPlanner, but I am concerned with too many repetitive messages
because VolcanoPlanner is usually instantiated per SQL query.
2. In the next release, (1) replace the warning with an error, (2) change
VolcanoPlanner.isTransformationRule as discussed above.



Пт, 28 мая 2021 г. в 21:27, Haisheng Yuan :

> Great, that is the correct way to change it and that should be the default
> implementation.
>
> On 2021/05/28 17:41:15, Vladimir Ozerov  wrote:
> > BTW, I tried to change the implementation to:
> >
> >  1: protected boolean isTransformationRule(VolcanoRuleCall match) {
> >  2:return match.getRule() instanceof TransformationRule;
> >  3: }
> >
> > It solved my problem - plans returned to normal. In the Apache Calcite
> > repo, only 4 tests in the TopDowOptTest class failed due to a minor
> > operator reordering.
> >
> > пт, 28 мая 2021 г. в 20:37, Vladimir Ozerov :
> >
> > > Hi Jinpeng,
> > >
> > > Thank you for the clarification. When I saw the code in question for
> the
> > > first time, my first thought was that it was perhaps designed for
> gradual
> > > migration. The main problem is that the current implementation discards
> > > parts of the plan *silently*, which might be difficult to spot. I
> > > only spotted the problem in my specific case because I had ~100 tests
> with
> > > complex queries. Otherwise, I would happily proceed with the new rule
> > > without knowing that I lost important parts of the search space.
> > >
> > > That said, I think we can do the following:
> > >
> > >1. Emit a warning if or even throw an exception if the
> transformation
> > >rule produced a physical node. This should be trivial to implement
> - add an
> > >expected node type to VolcanoRuleCall (e.g., "logical", "physical",
> "any").
> > >The warning/exception should contain a proper fix suggestion - to
> override
> > >the VolcanoPlanner.isTransformationRule.
> > >2. Alternatively - do a breaking change. Apache Calcite doesn't
> have a
> > >major release cadence. It is normal practice in many products to do
> > >breaking changes in minor releases. Even popular products like
> Mongo or
> > >DataStax do it regularly. We may inform the user in the first
> release and
> > >change to "rule instanceof TransformationRule" in the next release.
> > >
> > > Does it make sense?
> > >
> > > Regards,
> > > Vladimir.
> > >
> > > пт, 28 мая 2021 г. в 19:33, Jinpeng Wu :
> > >
> > >> Hi, Vladimir. Good catch! There could be some improvements here.
> > >>
> > >> Actually, this problem was discovered early when the top-down rule
> driver
> > >> was designed. At that time, no rule was annotated as
> "TransformationRule".
> > >> Moreover, it is impossible to ask every calcite user who designed
> their
> > >> own
> > >> rules to annotate the existing code. So the top-down rule driver was
> > >> designed so that it can:
> > >> 1. Work in chaos: even if there are no hints for rule types, it can
> still
> > >> work. Some opportunities may be lost, but NO failures, NO exceptions,
> and
> > >> NO worse than the original driver. People can migrate to the new
> driver
> > >> without concern.
> > >> 2. Be Improvable: Users can refactor their code, if they want, step by
> > >> step. As rule types become more and more accurate, the system achieves
> > >> more
> > >> and more benefits
> > >> 3. Be easy to customize: the default implementation is designed for
> the
> > >> most common cases, so that most users can benefit from it without much
> > >> effort. But it is not possible to fulfill all requirements as
> different
> > >> systems could have very different patterns to define logical and
> > >> physical. That's why the isTransformationRule method is put in
> > >> VolcanoPlanner and marked as protected: overwriting it can be very
> simple.
> > >>
> > >> Moreover, losing some "derive" opportunities is not as serious as
> > >> imagination. As

Re: Top-down optimizer cannot explore the search space because physical rule is treated as transformation rule

2021-05-30 Thread Vladimir Ozerov
Hi Jinpeng,

Do you mean the whole change or the error part only?

My concern is that if we change the implementation of
VolcanoPlanner.isTransformationRule, then some transformation rules that
are not marked as "implements TransformationRule" will be treated as
implementation rules, which may lead to some other hidden negative
consequences.

Ease of upgrade and predictable behavior is often in conflict with each
other when planning migration paths. I am not insisting on the error, but
personally, I am more comfortable with products that fail fast, forcing me
to do the right things rather as early as possible.

Regards,
Vladimir.

[1]
https://github.com/apache/calcite/blob/calcite-1.26.0/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java#L96-L100

вс, 30 мая 2021 г. в 10:45, Jinpeng Wu :

> Hi.
>
> Warnings and Fixing isTransformationRule are good solutions. But I am still
> concerned about reporting an error. It will force users to do a large
> refactoring on existing codes before they can migrate to the new rule
> driver. Any refactoring can be risky, especially for those critical
> services. It leaves those systems no choice but to keep using old versions
> of calcite. However, we usually can still get a good plan even without
> correct rule types. I don't think it worthwhile to introduce that change.
>
> Thanks
> Jinpeng Wu
>
> On Sun, May 30, 2021 at 5:19 AM Vladimir Ozerov 
> wrote:
>
> > Great. Does the community have any objections to the following fix?
> > 1. Add a flag to a rule call instance with the expected node type. In the
> > case of a mismatch, we will print a warning once per rule per JVM
> instance
> > to avoid too many messages. Alternatively, we may print a warning per
> rule
> > per VolcanoPlanner, but I am concerned with too many repetitive messages
> > because VolcanoPlanner is usually instantiated per SQL query.
> > 2. In the next release, (1) replace the warning with an error, (2) change
> > VolcanoPlanner.isTransformationRule as discussed above.
> >
> >
> >
> > Пт, 28 мая 2021 г. в 21:27, Haisheng Yuan :
> >
> > > Great, that is the correct way to change it and that should be the
> > default
> > > implementation.
> > >
> > > On 2021/05/28 17:41:15, Vladimir Ozerov  wrote:
> > > > BTW, I tried to change the implementation to:
> > > >
> > > >  1: protected boolean isTransformationRule(VolcanoRuleCall match) {
> > > >  2:return match.getRule() instanceof TransformationRule;
> > > >  3: }
> > > >
> > > > It solved my problem - plans returned to normal. In the Apache
> Calcite
> > > > repo, only 4 tests in the TopDowOptTest class failed due to a minor
> > > > operator reordering.
> > > >
> > > > пт, 28 мая 2021 г. в 20:37, Vladimir Ozerov :
> > > >
> > > > > Hi Jinpeng,
> > > > >
> > > > > Thank you for the clarification. When I saw the code in question
> for
> > > the
> > > > > first time, my first thought was that it was perhaps designed for
> > > gradual
> > > > > migration. The main problem is that the current implementation
> > discards
> > > > > parts of the plan *silently*, which might be difficult to spot. I
> > > > > only spotted the problem in my specific case because I had ~100
> tests
> > > with
> > > > > complex queries. Otherwise, I would happily proceed with the new
> rule
> > > > > without knowing that I lost important parts of the search space.
> > > > >
> > > > > That said, I think we can do the following:
> > > > >
> > > > >1. Emit a warning if or even throw an exception if the
> > > transformation
> > > > >rule produced a physical node. This should be trivial to
> implement
> > > - add an
> > > > >expected node type to VolcanoRuleCall (e.g., "logical",
> > "physical",
> > > "any").
> > > > >The warning/exception should contain a proper fix suggestion -
> to
> > > override
> > > > >the VolcanoPlanner.isTransformationRule.
> > > > >2. Alternatively - do a breaking change. Apache Calcite doesn't
> > > have a
> > > > >major release cadence. It is normal practice in many products to
> > do
> > > > >breaking changes in minor releases. Even popular products like
> > > Mongo or
> > > > >DataStax do it regularly. We may infor

Possibly incorrect assertion in the TopDownRuleDriver.DeriveTrait.derive

2021-06-06 Thread Vladimir Ozerov
Hi,

When doing a trait derivation in the non-OMAKASE mode, the following lines
of code are invoked:
1: RelSubset relSubset = planner.register(newRel, rel);
2: assert relSubset.set == planner.getSubset(rel).set;

The assertion on the second line may fail because the "newRel" is assigned
not the "rel" set, but "rel" *canonical set*, which might be different.

As a workaround, we may change the derive mode to OMAKASE. In this case, we
do not hit the assertion and planning completes successfully.

Shouldn't we remove the assertion above?

Regards,
Vladimir.


Re: Top-down optimizer cannot explore the search space because physical rule is treated as transformation rule

2021-06-13 Thread Vladimir Ozerov
Hi,

I tried to apply different approaches, but eventually, I failed to achieve
my goals. It seems that the current implementation cannot handle the
required scenario, as explained below.

Consider the following tree:
LogicalAggregate1[group=[b,c]]
  LogicalAggregate2[group=[a,b,c]]
LogicalInput

I want to find the plan to do these two aggregations without an exchange in
between because they may have compatible distributions. Example:
PhysicalAggregate1[group=[b,c]] // SHARDED[b,c]
  PhysicalAggregate2[group=[a,b,c]] // SHARDED[b,c]
Exchange// SHARDED[b,c]
  PhysicalInput // SHARDED[?]

The fundamental problem is that it is impossible to save the optimization
request and resolve traits in the "derive" phase afterward. What we need is
to send the optimization request "SHARDED by [b,c] in any order" to
PhysicalAggregate2, and use it in the derive phase so that the new
PhysicalAggregate2 is created with [b,c] or [c,b], but strictly without
[a]. Unfortunately, this doesn't work because the nodes emitted from the
pass-through do not participate in the "derive" phase.

This could be fixed with a trivial change - to allow certain nodes emitted
from the "passThrough" to participate in "derive". We can do that using a
marker interface or an extension to a PhysicalRel interface. For example:
interface PhysicalRel {
boolean enforceDerive();
}

When set to "true", the node would not be added to the pass-through cache.
This way, we may use this node as *storage* for the optimization request.
When the "derive" is called later, we know both the parent requirements and
the child traits. This would be sufficient to solve my problem. I already
tried to do this by disabling the pass-through cache completely and
confirmed that the required plan is found.

Do you have any objections to such a change?

Regards,
Vladimir.


вс, 30 мая 2021 г. в 12:54, Jinpeng Wu :

> Hi, Vladimir. I mean the error part only.  I am comfortable with the other
> changes.
>
> If changing isTransformationRule could have unexpected consequences, one
> possible way is reserving current logic and only adding a newline checking
> the "implements TransformationRule''. Even though we remove the original
> logic completely, users who prefer legacy logic to avoid risks can
> overwrite the method by simply copying several lines of code. That's
> totally acceptable. But if errors are issued, that's no longer a choice.
>
> In any case, if errors should be reported, we should provide an option to
> suppress the errors.
>
> Thanks,
> Jinpeng
>
> On Sun, May 30, 2021 at 4:59 PM Vladimir Ozerov 
> wrote:
>
> > Hi Jinpeng,
> >
> > Do you mean the whole change or the error part only?
> >
> > My concern is that if we change the implementation of
> > VolcanoPlanner.isTransformationRule, then some transformation rules that
> > are not marked as "implements TransformationRule" will be treated as
> > implementation rules, which may lead to some other hidden negative
> > consequences.
> >
> > Ease of upgrade and predictable behavior is often in conflict with each
> > other when planning migration paths. I am not insisting on the error, but
> > personally, I am more comfortable with products that fail fast, forcing
> me
> > to do the right things rather as early as possible.
> >
> > Regards,
> > Vladimir.
> >
> > [1]
> >
> >
> https://github.com/apache/calcite/blob/calcite-1.26.0/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java#L96-L100
> >
> > вс, 30 мая 2021 г. в 10:45, Jinpeng Wu :
> >
> > > Hi.
> > >
> > > Warnings and Fixing isTransformationRule are good solutions. But I am
> > still
> > > concerned about reporting an error. It will force users to do a large
> > > refactoring on existing codes before they can migrate to the new rule
> > > driver. Any refactoring can be risky, especially for those critical
> > > services. It leaves those systems no choice but to keep using old
> > versions
> > > of calcite. However, we usually can still get a good plan even without
> > > correct rule types. I don't think it worthwhile to introduce that
> change.
> > >
> > > Thanks
> > > Jinpeng Wu
> > >
> > > On Sun, May 30, 2021 at 5:19 AM Vladimir Ozerov 
> > > wrote:
> > >
> > > > Great. Does the community have any objections to the following fix?
> > > > 1. Add a flag to a rule call instance with the expected node type. In
> > the
> > > > case of a mismatch, we wi

Re: Top-down optimizer cannot explore the search space because physical rule is treated as transformation rule

2021-06-13 Thread Vladimir Ozerov
Please disregard this email. I sent it to the wrong thread, sorry.

вс, 13 июн. 2021 г. в 19:56, Vladimir Ozerov :

> Hi,
>
> I tried to apply different approaches, but eventually, I failed to achieve
> my goals. It seems that the current implementation cannot handle the
> required scenario, as explained below.
>
> Consider the following tree:
> LogicalAggregate1[group=[b,c]]
>   LogicalAggregate2[group=[a,b,c]]
> LogicalInput
>
> I want to find the plan to do these two aggregations without an exchange
> in between because they may have compatible distributions. Example:
> PhysicalAggregate1[group=[b,c]] // SHARDED[b,c]
>   PhysicalAggregate2[group=[a,b,c]] // SHARDED[b,c]
> Exchange// SHARDED[b,c]
>   PhysicalInput // SHARDED[?]
>
> The fundamental problem is that it is impossible to save the optimization
> request and resolve traits in the "derive" phase afterward. What we need is
> to send the optimization request "SHARDED by [b,c] in any order" to
> PhysicalAggregate2, and use it in the derive phase so that the new
> PhysicalAggregate2 is created with [b,c] or [c,b], but strictly without
> [a]. Unfortunately, this doesn't work because the nodes emitted from the
> pass-through do not participate in the "derive" phase.
>
> This could be fixed with a trivial change - to allow certain nodes emitted
> from the "passThrough" to participate in "derive". We can do that using a
> marker interface or an extension to a PhysicalRel interface. For example:
> interface PhysicalRel {
> boolean enforceDerive();
> }
>
> When set to "true", the node would not be added to the pass-through cache.
> This way, we may use this node as *storage* for the optimization request.
> When the "derive" is called later, we know both the parent requirements and
> the child traits. This would be sufficient to solve my problem. I already
> tried to do this by disabling the pass-through cache completely and
> confirmed that the required plan is found.
>
> Do you have any objections to such a change?
>
> Regards,
> Vladimir.
>
>
> вс, 30 мая 2021 г. в 12:54, Jinpeng Wu :
>
>> Hi, Vladimir. I mean the error part only.  I am comfortable with the other
>> changes.
>>
>> If changing isTransformationRule could have unexpected consequences, one
>> possible way is reserving current logic and only adding a newline checking
>> the "implements TransformationRule''. Even though we remove the original
>> logic completely, users who prefer legacy logic to avoid risks can
>> overwrite the method by simply copying several lines of code. That's
>> totally acceptable. But if errors are issued, that's no longer a choice.
>>
>> In any case, if errors should be reported, we should provide an option to
>> suppress the errors.
>>
>> Thanks,
>> Jinpeng
>>
>> On Sun, May 30, 2021 at 4:59 PM Vladimir Ozerov 
>> wrote:
>>
>> > Hi Jinpeng,
>> >
>> > Do you mean the whole change or the error part only?
>> >
>> > My concern is that if we change the implementation of
>> > VolcanoPlanner.isTransformationRule, then some transformation rules that
>> > are not marked as "implements TransformationRule" will be treated as
>> > implementation rules, which may lead to some other hidden negative
>> > consequences.
>> >
>> > Ease of upgrade and predictable behavior is often in conflict with each
>> > other when planning migration paths. I am not insisting on the error,
>> but
>> > personally, I am more comfortable with products that fail fast, forcing
>> me
>> > to do the right things rather as early as possible.
>> >
>> > Regards,
>> > Vladimir.
>> >
>> > [1]
>> >
>> >
>> https://github.com/apache/calcite/blob/calcite-1.26.0/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java#L96-L100
>> >
>> > вс, 30 мая 2021 г. в 10:45, Jinpeng Wu :
>> >
>> > > Hi.
>> > >
>> > > Warnings and Fixing isTransformationRule are good solutions. But I am
>> > still
>> > > concerned about reporting an error. It will force users to do a large
>> > > refactoring on existing codes before they can migrate to the new rule
>> > > driver. Any refactoring can be risky, especially for those critical
>> > > services. It leaves those systems no choice but to keep using old
>> > versions
>> > > of calcite. However, we usually can 

Re: Trait propagation guidelines

2021-06-13 Thread Vladimir Ozerov
Hi,

I tried to apply different approaches, but eventually, I failed to achieve
my goals. It seems that the current implementation cannot handle the
required scenario, as explained below.

Consider the following tree:
LogicalAggregate1[group=[b,c]]
  LogicalAggregate2[group=[a,b,c]]
LogicalInput

I want to find the plan to do these two aggregations without an exchange in
between because they may have compatible distributions. Example:
PhysicalAggregate1[group=[b,c]] // SHARDED[b,c]
  PhysicalAggregate2[group=[a,b,c]] // SHARDED[b,c]
Exchange// SHARDED[b,c]
  PhysicalInput // SHARDED[?]

The fundamental problem is that it is impossible to save the optimization
request and resolve traits in the "derive" phase afterward. What we need is
to send the optimization request "SHARDED by [b,c] in any order" to
PhysicalAggregate2, and use it in the derive phase so that the new
PhysicalAggregate2 is created with [b,c] or [c,b], but strictly without
[a]. Unfortunately, this doesn't work because the nodes emitted from the
pass-through do not participate in the "derive" phase.

This could be fixed with a trivial change - to allow certain nodes emitted
from the "passThrough" to participate in "derive". We can do that using a
marker interface or an extension to a PhysicalRel interface. For example:
interface PhysicalRel {
boolean enforceDerive();
}

When set to "true", the node would not be added to the pass-through cache.
This way, we may use this node as *storage* for the optimization request.
When the "derive" is called later, we know both the parent requirements and
the child traits. This would be sufficient to solve my problem. I already
tried to do this by disabling the pass-through cache completely and
confirmed that the required plan is found.

Do you have any objections to such a change?

Regards,
Vladimir.

сб, 29 мая 2021 г. в 11:59, Vladimir Ozerov :

> Hi Haisheng, Jinpeng
>
> I think we are more or less on the same page:
>
>1. The current implementation of Apache Calcite may generate wasteful
>alternatives because rules lack the optimization context.
>2. But the actual impact on efficiency is not clear.
>
> The (2) is essential to understand whether my efforts make any practical
> sense. And so far, I have only a vague common sense and some simple
> examples in mind, which is not sufficient to make any claims.
>
> Nevertheless, I've checked the source code of the original Columbia
> optimizer. I was wrong in my original claim that Columbia doesn't pass
> optimization context to rules. It does [1]. The context consists of
> required traits and cost budget. In Apache Calcite terms, the context is
> passed to both "RelRule.matches" and "RelRule.onMatch", so that the rule
> may decide on the optimization strategy based on parent request. This is
> exactly what I was trying to achieve in my system with some hacks around
> derive/passThrough.
>
> Regarding the example with join, my proposal is not likely to make any
> difference because the tables are not co-located on the join key, and hence
> join may emit several distributions. Consider the different situation -
> data is already collocated. Without the context, I will emit both 1-phase
> and 2-phase aggregates because I do not know which distributions are
> available below. With the context available, I can collect propagate
> promising optimization requests from Aggregate rules (1-phase, 2-phase).
> Then wait for input optimization and check what is returned. If only
> [dist=a] is returned, I can skip the 2-phase aggregate completely.
> Aggregate[group=a]
>   Join[foo.a=bar.b]
> Input(foo, dist=a)
> Input(bar, dist=b)
>
> Another possible use case is join on several keys. By issuing a
> context-aware optimization request [dist a1] from Aggregate to Join, we
> can establish tight cost bounds on Aggregate and Join equivalence groups
> very early so that all other options (broadcasts, sharding in [a1,a2], ...)
> would be pruned without even entering MEMO.
> Aggregate[group=a1]
>   Join[foo.a1=bar.b1 AND foo.a2=bar.b2]
> Input(foo, dist=a1)
> Input(bar, dist=b2)
>
> As far as Jinpeng's example with logical multi-phase aggregates - I think
> this is a great example of why logical split might be useful. Thank you for
> that. This reminded me about another concerning use case. Consider an
> Aggregate on top of a UnionAll:
> LogicalAggregate[group=a, COUNT(b)]
>   UnionAll
> Input1
> Input2
>
> With Calcite rules, we may push the aggregate down:
> LogicalAggregate[group=a, SUM(COUNT)]
>   UnionAll
> LogicalAggregate[group=a, COUNT(b)] // <-- Possible exchange here
>   Input1
&g

Re: Possibly incorrect assertion in the TopDownRuleDriver.DeriveTrait.derive

2021-06-13 Thread Vladimir Ozerov
Thanks, I created an issue [1] to improve the assertion.

[1] https://issues.apache.org/jira/browse/CALCITE-4650

пн, 7 июн. 2021 г. в 23:30, Haisheng Yuan :

> > Shouldn't we remove the assertion above?
> Perhaps.
>
> Or perhaps the rel2Subset mapping is not up to date.
>
> Regards,
> Haisheng Yuan
>
> On 2021/06/06 13:09:16, Vladimir Ozerov  wrote:
> > Hi,
> >
> > When doing a trait derivation in the non-OMAKASE mode, the following
> lines
> > of code are invoked:
> > 1: RelSubset relSubset = planner.register(newRel, rel);
> > 2: assert relSubset.set == planner.getSubset(rel).set;
> >
> > The assertion on the second line may fail because the "newRel" is
> assigned
> > not the "rel" set, but "rel" *canonical set*, which might be different.
> >
> > As a workaround, we may change the derive mode to OMAKASE. In this case,
> we
> > do not hit the assertion and planning completes successfully.
> >
> > Shouldn't we remove the assertion above?
> >
> > Regards,
> > Vladimir.
> >
>


Re: Trait propagation guidelines

2021-06-14 Thread Vladimir Ozerov
Hi Haisheng,

The observation is that parent operators sometimes do not know the exact
traits they will have for the given child traits. Several examples:
1. PhysicalAggregate1 may request both HASH[b,c] or HASH[c,b]. Contrary to
the default Apache Calcite implementation, in many systems, these are two
different distributions - which one should I request? To make things worse,
some operators may have strict requirements to the order (Join, Union),
whilst others do not care about the order (Aggregate, Window).
2. In some systems, the distribution may also define the distribution
function, e.g., a number of shards. A UNION DISTINCT of two tables with the
same sharding key, but the different numbers of shards must yield an
exchange. The parent operator cannot know the number of shards of the input
in advance and cannot define the proper trait set in the "passThrough"
method.

We will miss the optimization opportunity in all these cases unless we can
clarify the real traits in the "derive" phase. But to do this, we need to
know the original optimization request.

Regards,
Vladimir.


вс, 13 июн. 2021 г. в 22:17, Haisheng Yuan :

> How does it relate with "derive" to get the desired plan?
>
> Initially PhysicalAggregate1 requests HASH[b,c], PhysicalAggregate2
> requests HASH[a,b,c]. PhysicalAggregate2 is called on "passthrough" by
> passing HASH[b,c], then generate another PhysicalAggregate2 with trait
> HASH[b,c]. You don't need the involvement of "derive".
>
> Haisheng Yuan
>
> On 2021/06/13 16:58:53, Vladimir Ozerov  wrote:
> > Hi,
> >
> > I tried to apply different approaches, but eventually, I failed to
> achieve
> > my goals. It seems that the current implementation cannot handle the
> > required scenario, as explained below.
> >
> > Consider the following tree:
> > LogicalAggregate1[group=[b,c]]
> >   LogicalAggregate2[group=[a,b,c]]
> > LogicalInput
> >
> > I want to find the plan to do these two aggregations without an exchange
> in
> > between because they may have compatible distributions. Example:
> > PhysicalAggregate1[group=[b,c]] // SHARDED[b,c]
> >   PhysicalAggregate2[group=[a,b,c]] // SHARDED[b,c]
> > Exchange// SHARDED[b,c]
> >   PhysicalInput // SHARDED[?]
> >
> > The fundamental problem is that it is impossible to save the optimization
> > request and resolve traits in the "derive" phase afterward. What we need
> is
> > to send the optimization request "SHARDED by [b,c] in any order" to
> > PhysicalAggregate2, and use it in the derive phase so that the new
> > PhysicalAggregate2 is created with [b,c] or [c,b], but strictly without
> > [a]. Unfortunately, this doesn't work because the nodes emitted from the
> > pass-through do not participate in the "derive" phase.
> >
> > This could be fixed with a trivial change - to allow certain nodes
> emitted
> > from the "passThrough" to participate in "derive". We can do that using a
> > marker interface or an extension to a PhysicalRel interface. For example:
> > interface PhysicalRel {
> > boolean enforceDerive();
> > }
> >
> > When set to "true", the node would not be added to the pass-through
> cache.
> > This way, we may use this node as *storage* for the optimization request.
> > When the "derive" is called later, we know both the parent requirements
> and
> > the child traits. This would be sufficient to solve my problem. I already
> > tried to do this by disabling the pass-through cache completely and
> > confirmed that the required plan is found.
> >
> > Do you have any objections to such a change?
> >
> > Regards,
> > Vladimir.
> >
> > сб, 29 мая 2021 г. в 11:59, Vladimir Ozerov :
> >
> > > Hi Haisheng, Jinpeng
> > >
> > > I think we are more or less on the same page:
> > >
> > >1. The current implementation of Apache Calcite may generate
> wasteful
> > >alternatives because rules lack the optimization context.
> > >2. But the actual impact on efficiency is not clear.
> > >
> > > The (2) is essential to understand whether my efforts make any
> practical
> > > sense. And so far, I have only a vague common sense and some simple
> > > examples in mind, which is not sufficient to make any claims.
> > >
> > > Nevertheless, I've checked the source code of the original Columbia
> > > optimizer. I was wrong in my original claim that Columbia doesn't pass
> > > optimization co

Re: Trait propagation guidelines

2021-06-25 Thread Vladimir Ozerov
ystem, what collation(s) do you request?
> Just request either one. I already stated in the email [1], but seems like
> you missed the 5th paragraph.
>
> > 2. In some systems, the distribution may also define the distribution
> > function, e.g., a number of shards. A UNION DISTINCT of two tables with
> the
> > same sharding key, but the different numbers of shards must yield an
> > exchange. The parent operator cannot know the number of shards of the
> input
> > in advance and cannot define the proper trait set in the "passThrough"
> > method.
> The parent operator doesn't need to know what number of shards to request,
> just request hash distribution with shard number 0 or -1 or what ever to
> indicate shard number not decided yet. Later the child operator will tell
> parent operator the exact distribution through "derive".
>
> In Alibaba MaxCompute, we have customized hash distribution, which
> contains number of buckets, hash function, null collation, we also support
> range distribution, which contains range bucket boundaries. All of these
> can work under current framework. With all that being said, distribution is
> nothing special than collation, it all depends on whether you design the
> operator "passthrough" and "derive" strategy correctly.
>
> [1]
> https://lists.apache.org/thread.html/r36b25cbe4ca05fb1262c432ad9103f4126b654698481fca0d2a01fe7%40%3Cdev.calcite.apache.org%3E
>
> Thanks,
> Haisheng Yuan
>
> On 2021/06/14 08:26:31, Vladimir Ozerov  wrote:
> > Hi Haisheng,
> >
> > The observation is that parent operators sometimes do not know the exact
> > traits they will have for the given child traits. Several examples:
> > 1. PhysicalAggregate1 may request both HASH[b,c] or HASH[c,b]. Contrary
> to
> > the default Apache Calcite implementation, in many systems, these are two
> > different distributions - which one should I request? To make things
> worse,
> > some operators may have strict requirements to the order (Join, Union),
> > whilst others do not care about the order (Aggregate, Window).
> > 2. In some systems, the distribution may also define the distribution
> > function, e.g., a number of shards. A UNION DISTINCT of two tables with
> the
> > same sharding key, but the different numbers of shards must yield an
> > exchange. The parent operator cannot know the number of shards of the
> input
> > in advance and cannot define the proper trait set in the "passThrough"
> > method.
> >
> > We will miss the optimization opportunity in all these cases unless we
> can
> > clarify the real traits in the "derive" phase. But to do this, we need to
> > know the original optimization request.
> >
> > Regards,
> > Vladimir.
> >
> >
> > вс, 13 июн. 2021 г. в 22:17, Haisheng Yuan :
> >
> > > How does it relate with "derive" to get the desired plan?
> > >
> > > Initially PhysicalAggregate1 requests HASH[b,c], PhysicalAggregate2
> > > requests HASH[a,b,c]. PhysicalAggregate2 is called on "passthrough" by
> > > passing HASH[b,c], then generate another PhysicalAggregate2 with trait
> > > HASH[b,c]. You don't need the involvement of "derive".
> > >
> > > Haisheng Yuan
> > >
> > > On 2021/06/13 16:58:53, Vladimir Ozerov  wrote:
> > > > Hi,
> > > >
> > > > I tried to apply different approaches, but eventually, I failed to
> > > achieve
> > > > my goals. It seems that the current implementation cannot handle the
> > > > required scenario, as explained below.
> > > >
> > > > Consider the following tree:
> > > > LogicalAggregate1[group=[b,c]]
> > > >   LogicalAggregate2[group=[a,b,c]]
> > > > LogicalInput
> > > >
> > > > I want to find the plan to do these two aggregations without an
> exchange
> > > in
> > > > between because they may have compatible distributions. Example:
> > > > PhysicalAggregate1[group=[b,c]] // SHARDED[b,c]
> > > >   PhysicalAggregate2[group=[a,b,c]] // SHARDED[b,c]
> > > > Exchange// SHARDED[b,c]
> > > >   PhysicalInput // SHARDED[?]
> > > >
> > > > The fundamental problem is that it is impossible to save the
> optimization
> > > > request and resolve traits in the "derive" phase afterward. What we
> need
> > > is
> > > > to send the optimization request "SHARDED by [b,c] in any 

Re: [ANNOUNCE] New committer: Vladimir Ozerov

2021-06-25 Thread Vladimir Ozerov
Thank you, everybody!

While I enjoy hacking Calcite's codebase, my colleagues and I observe a
consistent signal from the "field" - lack of documentation blocks many
attempts to integrate Apache Calcite. Engineers simply get lost with no
idea on how to move forward. So I more and more believe that (usually
boring) documentation-related efforts might boost Apache Calcite adoption
dramatically. Hopefully, we will find time to invest in it.

Regards,
Vladimir.

пт, 25 июн. 2021 г. в 09:52, Fan Liya :

> Congratulations, Vladimir!
> Thanks for the good articles.
>
> Best,
> Liya Fan
>
> On Fri, Jun 25, 2021 at 9:07 AM Julian Hyde 
> wrote:
>
> > Welcome, Vladimir!
> >
> > > On Jun 24, 2021, at 6:00 PM, Albert  wrote:
> > >
> > > Congrats.
> > > just found the blog:
> https://www.querifylabs.com/author/vladimir-ozerov
> > >
> > > On Thu, Jun 24, 2021 at 2:27 PM Alessandro Solimando <
> > > alessandro.solima...@gmail.com> wrote:
> > >
> > >> Congratulations Vladimir, well deserved, I had the chance to read some
> > >> of the blog posts and I have appreciated them very much.
> > >>
> > >> Best regards,
> > >> Alessandro
> > >>
> > >> On Thu, 24 Jun 2021 at 07:58, Viliam Durina 
> > wrote:
> > >>>
> > >>> Congratulations!
> > >>>
> > >>> Viliam
> > >>>
> > >>> On Thu, 24 Jun 2021 at 06:58, Forward Xu 
> > wrote:
> > >>>
> > >>>> Congratulations!
> > >>>>
> > >>>>
> > >>>> Best,
> > >>>>
> > >>>> Forward
> > >>>>
> > >>>> Danny Chan  于2021年6月24日周四 上午11:51写道:
> > >>>>
> > >>>>> Congrats, Vladimir!
> > >>>>>
> > >>>>> Best,
> > >>>>> Danny Chan
> > >>>>>
> > >>>>> Yanjing Wang  于2021年6月24日周四 上午11:41写道:
> > >>>>>
> > >>>>>> Congrats, Vladimir!
> > >>>>>>
> > >>>>>> Roman Kondakov  于2021年6月24日周四
> > >> 上午11:22写道:
> > >>>>>>
> > >>>>>>> Congratulations, Vladimir!
> > >>>>>>>
> > >>>>>>> Roman Kondakov
> > >>>>>>>
> > >>>>>>> On 24.06.2021 12:23, 段雄 wrote:
> > >>>>>>>> Congratulations!
> > >>>>>>>>
> > >>>>>>>> XING JIN  于2021年6月24日周四 上午10:21写道:
> > >>>>>>>>
> > >>>>>>>>> Congratulations ~
> > >>>>>>>>>
> > >>>>>>>>> Best,
> > >>>>>>>>> Jin
> > >>>>>>>>>
> > >>>>>>>>> guangyuan wang  于2021年6月24日周四
> > >> 上午9:50写道:
> > >>>>>>>>>
> > >>>>>>>>>> Congratulations!
> > >>>>>>>>>>
> > >>>>>>>>>> Francis Chuang  于2021年6月24日周四
> > >>>> 上午6:39写道:
> > >>>>>>>>>>
> > >>>>>>>>>>> Congrats, Vladimir!
> > >>>>>>>>>>>
> > >>>>>>>>>>> Francis
> > >>>>>>>>>>>
> > >>>>>>>>>>> On 24/06/2021 7:48 am, Haisheng Yuan wrote:
> > >>>>>>>>>>>> Congratulations and thanks for your contributions,
> > >> Vladimir!
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Regards,
> > >>>>>>>>>>>> Haisheng
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On 2021/06/23 21:34:40, Stamatis Zampetakis <
> > >> zabe...@gmail.com
> > >>>>>
> > >>>>>>>>> wrote:
> > >>>>>>>>>>>>> Apache Calcite's Project Management Committee (PMC) has
> > >>>> invited
> > >>>>>>>>>> Vladimir
> > >>>>>>>>>>>>> Ozerov to
> > >>>>>

Re: DISCUSS: merge calcite-avatica and calcite repositories

2021-11-09 Thread Vladimir Ozerov
+1 for a single repo.

Вт, 9 нояб. 2021 г. в 19:22, Vladimir Sitnikov :

> Michael> is not proposing to change the
> Michael>structure of modules within both projects, merely to have the code
> for
> Michael>both in a single repository.
>
> I propose to integrate them into a single build, and keep the set of the
> published jars.
> However, the modules and dependency structure could be kept as is.
>
> We might want to rename folders like
> calcite-core
> calcite-linq4j
> ..
> avatica-core
> avatica-server
>
> However, I am not sure it is that important to discuss folder names now.
> The idea is that as you "open Calcite in IDE, you see both Avatica and
> Calcite modules"
>
> Michael>Is there any reason we couldn't have a separate release schedule if
> Michael>both projects are in the same repository?
>
> A different schedule means Calcite must support at least two different
> Avatica versions.
> In other words, if we allow clients to update Avatica at their will, then
> we should allow them building Calcite with different Avatica versions,
> which implies Calcite test code should succeed for multiple different
> Avatica vesions.
>
> That makes it harder to write tests: we have to execute tests with two
> different Avatica releases (or even more than two).
>
> There are at least two sources for complexity:
>
> a) We have to write tests that tolerate multiple versions. For instance,
> "if (avatica.18+) {...}" and so on.
> That is not really trivial, especially taking into account some of the
> tests are created with non-yet-popular
> technologies like Quidem where you can't really google solutions. So the
> "trivial" task of "making a test to expect two possible outcomes"
> becomes less trivial as you try to pass the version from GitHub Action to
> Gradle to JUnit to Quidem to no-one-knows-which class.
> If we support one Avatica version only, that is not needed. We just patch
> the test in Avatica and Calcite and that is it.
> Single repo avoids "Gradle vs Quidem" dance.
>
> b) If we claim that we support 5 different Guava versions, 3 different JDK
> versions, 2 different Avatica versions,
> then we have to execute 5*3*2 = 30 combinations of the tests.
> That is not really a full matrix, however, things get way easier if we
> support one Avatica version only.
> The amount of tests we need to do during a proper release is much less, and
> it is easier to commit
> changes that touch Avatica and Calcite at the same time.
>
>
> Vladimir
>


Re: [DISCUSS] Apache Calcite Online Meetup January 2022

2021-11-18 Thread Vladimir Ozerov
+ 1

>From our side (Querify Labs), we may present one of the difficult use cases
we faced this year, such as large-scale join order planning, experience
with Cascades integration, planning for heterogeneous systems,
advanced statistics management, etc. We may need some time to figure out
what can or cannot be presented due to NDAs. If the dates are known, we
will start figuring out the possible list of topics.

I already presented at the previous meetup. So if there are more proposals
than slots, please prefer other proposals, and we will move our talk to
future meetups.

Regards,
Vladimir.

чт, 18 нояб. 2021 г. в 00:16, Alessandro Solimando <
alessandro.solima...@gmail.com>:

> I'd be interested too!
>
> Il Mer 17 Nov 2021, 19:26 Jacques Nadeau  ha scritto:
>
> > Great idea. +1 from me.
> >
> > On Wed, Nov 17, 2021 at 8:22 AM Stamatis Zampetakis 
> > wrote:
> >
> > > Hi all,
> > >
> > > Last meetup [1] was roughly 10 months ago. How about organising another
> > one
> > > around mid January 2022?
> > >
> > > Organising an in-person event would be nice but I think it is still a
> bit
> > > complicated so I was gonna suggest a virtual one (via Zoom meeting).
> > >
> > > The format could be ~1h for presentations and ~1h for open discussion.
> > >
> > > Are the people willing to give a talk around Calcite?
> > >
> > > What do people think in terms of timing?
> > >
> > > Best,
> > > Stamatis
> > >
> > > [1] https://www.meetup.com/Apache-Calcite/
> > >
> >
>


Re: RelNode semantic evolution vs adapter implementations

2021-12-13 Thread Vladimir Ozerov
Practically, Apache Calcite doesn’t have public API. Real projects usually
override not only public extension points, but also internals, such as
SqlValidatorImpl or VolcanoPlanner.

Given the flexibility that Apache Calcite provides, this lack of public API
surface is not necessarily a bad thing, since you may change almost
everything without forking the project. On the other hand, almost any
change to any part of Calcite code base may cause regressions in downstream
projects.

I am not sure there is an ideal way of versioning rels because even if you
do so, some direct or indirect changes (metadata, rules, validator, etc)
may cause changes to plans around the given RelNode in the downstream
project.

Maybe instead of providing versions manually, we may expose some sort of
generated signatures for different components - rels, rules, metadata
handlers, golden plans, etc. Then, we may provide the ability to verify
expected and actual signatures, e.g., using some code generation: user
invokes a command with the list of interesting classes, and gets the source
code of the test that he includes into his project and CI. Now, when the
user migrates to the new version, tests for changed entities will fail, and
user will investigate the associated changes.

The main difference s from the original proposal:
1. Wider scope, because validation of rels is often not sufficient.
2. Automation, because as implement or you cannot always predict the effect
of your changes (e.g., metadata).

WDYT?

Вт, 14 дек. 2021 г. в 01:49, Konstantin Orlov :

> > The case there is that "correlation variables" are added to the Logical*
> nodes (e.g. LogicalFilter, LogicalProject).
>
> BTW, "correlation variables" were added to a LogicalFilter in CALCITE-816.
> So what is wrong with CALCITE-3183?
>
> вт, 14 дек. 2021 г. в 01:24, Konstantin Orlov :
>
> > Vladimir, could you please clarify in what way the PR#2623 changes
> > the semantics?
> >
> > The correlated project is already possible in the master. The
> MongoProject
> > already discards variablesSet (simply because it's currently not stored
> > for
> > project node) and either fails or returns invalid results. This behavior
> > (alas,
> > incorrect) will be preserved after this patch.
> >
> > пн, 13 дек. 2021 г. в 17:55, Vladimir Sitnikov <
> > sitnikov.vladi...@gmail.com>:
> >
> >> Hi,
> >>
> >> It turns out Calcite's nodes can change semantics without notification
> for
> >> the end-users.
> >>
> >> Here are the release notes for Calcite 1.21, and it says **nothing**
> like
> >> "Ensure you handle Filter#variablesSet in case you implement Filter or
> in
> >> case you transform LogicalFilter in your rules"
> >> https://calcite.apache.org/news/2019/09/11/release-1.21.0/
> >>
> >> On top of that, the in-core adapters fail to handle that properly. For
> >> example, MongoFilter discards Filter#variablesSet.
> >>
> >> Can we please stop changing the semantics of RelNodes or can we have a
> >> better way to detect the changes in the client code?
> >>
> >> What if we add a "version" property to the corresponding RelNodes, and
> we
> >> increment it every time the semantic changes?
> >> Then client APIs could be coded like "ok, I'm prepared to handle Project
> >> v4, and Filter v5" (e.g. make "version" required when registering a
> rule),
> >> and there will be a runtime error in case Calcite generates Filter v6 in
> >> runtime.
> >>
> >> ---
> >>
> >> Sample case:
> >> CALCITE-3183 Trimming method for Filter rel uses wrong traitSet
> >> CALCITE-4913 Correlated variables in a select list are not deduplicated
> >>
> >> The case there is that "correlation variables" are added to the Logical*
> >> nodes (e.g. LogicalFilter, LogicalProject).
> >> Unfortunately, that change is hard to notice: there's no compilation
> >> failure, and there's no runtime error.
> >>
> >> The old client code just discards "correlation variables", and I guess
> it
> >> would result in wrong results or something like that.
> >> Silent wrong results is a really sad outcome from the database.
> >>
> >> CALCITE-4913 / PR#2623 adds Project#variablesSet property as well, and I
> >> guess it would in the same hidden semantic loss.
> >>
> >> Vladimir
> >>
> >
> >
> > --
> > Regards,
> > Konstantin Orlov
> >
>
>
> --
> Regards,
> Konstantin Orlov
>


Re: [Calcite Release] Preparing for 1.29.0 Release

2021-12-17 Thread Vladimir Ozerov
Hi,

Could you please also merge
https://github.com/apache/calcite/pull/2464 ? I guess it got stuck somehow,
as all the comments are addressed. This issue forced us to create a copy of
the AggregateUnionTransposeRule in our project.

Regards,
Vladimir

Сб, 18 дек. 2021 г. в 09:51, Rui Wang :

> Thanks Julian!
>
> Now we will just decide whether to merge CALCITE-4907 or not, then I will
> lock the main branch and start the release process.
>
>
> -Rui
>
> On Fri, Dec 17, 2021 at 10:29 PM Julian Hyde 
> wrote:
>
> > Rui, I have merged CALCITE-4946.
> >
> > > On Dec 17, 2021, at 10:21 PM, Rui Wang  wrote:
> > >
> > > Thanks Zhe.
> > >
> > > Hi Francesco,
> > >
> > > I left a comment in your PR. Can you take a look?
> > >
> > > -Rui
> > >
> > > On Fri, Dec 17, 2021 at 3:32 PM Zhe Hu  wrote:
> > >
> > >> Hi, Rui.
> > >> CALCITE-4860 will be fixed in next release, you can pass it currently.
> > >> Thanks for your work!
> > >>
> > >>
> > >> Best,
> > >> ZheHu
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >> On 12/18/2021 06:08,Francesco Gini wrote:
> > >> https://issues.apache.org/jira/browse/CALCITE-4907 is in review. I
> > believe
> > >> I have addressed all the comments. Let me know if I missed anything.
> > >>
> > >> On Fri, 17 Dec 2021 at 22:42, Jacques Nadeau 
> > wrote:
> > >>
> > >> I've merged CALCITE-4948 to mitigate the concerns around CALCITE-4898
> > and
> > >> maven repositories and category x licenses.
> > >>
> > >> On Fri, Dec 17, 2021 at 1:02 PM Rui Wang 
> wrote:
> > >>
> > >> For the list of 1.29 fix Jira, several Jira have been marked as
> resolved
> > >> since this Monday. I will do another pass today but probably not all
> > >> those
> > >> will make the 1.29.0 release.
> > >>
> > >>
> > >> -Rui
> > >>
> > >> On Fri, Dec 17, 2021 at 11:57 AM Ruben Q L  wrote:
> > >>
> > >> Hello,
> > >>
> > >> FYI https://issues.apache.org/jira/browse/CALCITE-4737 has been
> > >> merged.
> > >>
> > >> Regards,
> > >> Ruben
> > >>
> > >>
> > >> On Fri, Dec 17, 2021 at 7:40 PM Julian Hyde 
> > >> wrote:
> > >>
> > >> Rui,
> > >>
> > >> Even though https://issues.apache.org/jira/browse/CALCITE-4898 <
> > >> https://issues.apache.org/jira/browse/CALCITE-4898> is marked fixed,
> > >> there’s some fallout regarding maven repositories and category X.
> > >> That
> > >> needs to be sorted out before RC0. See [1].
> > >>
> > >> I’m working on getting
> > >> https://issues.apache.org/jira/browse/CALCITE-4946
> > >>  done today.
> > >> Please
> > >> wait for that.
> > >>
> > >> https://issues.apache.org/jira/browse/CALCITE-4885 <
> > >> https://issues.apache.org/jira/browse/CALCITE-4885> is a big
> > >> refactoring
> > >> and, though I’d love to get it in before the release, it isn’t going
> > >> to
> > >> happen. Don’t wait for it.
> > >>
> > >> I also see the following as open or in progress and fix-in-1.29:
> > >> * https://issues.apache.org/jira/browse/CALCITE-4872 <
> > >> https://issues.apache.org/jira/browse/CALCITE-4872> (Will Noble)
> > >> * https://issues.apache.org/jira/browse/CALCITE-4908 <
> > >> https://issues.apache.org/jira/browse/CALCITE-4908> (Woonsan Koo)
> > >> * https://issues.apache.org/jira/browse/CALCITE-4704 <
> > >> https://issues.apache.org/jira/browse/CALCITE-4704> (Stamatis)
> > >> * https://issues.apache.org/jira/browse/CALCITE-4737 <
> > >> https://issues.apache.org/jira/browse/CALCITE-4737> (Thomas Rebele)
> > >> * https://issues.apache.org/jira/browse/CALCITE-4860 <
> > >> https://issues.apache.org/jira/browse/CALCITE-4860> (Zhe Hu / Duan
> > >> Xiong)
> > >> * https://issues.apache.org/jira/browse/CALCITE-4907 <
> > >> https://issues.apache.org/jira/browse/CALCITE-4907> (Francesco Gini)
> > >>
> > >> What’s the status on those?
> > >>
> > >> Julian
> > >>
> > >> [1] https://github.com/apache/calcite/pull/2622 <
> > >> https://github.com/apache/calcite/pull/2622>
> > >>
> > >> PS It feels churlish to bring this up, but in time-honored tradition
> > >> the
> > >> email subject has been ‘[DISCUSS] Towards Calcite x.y.z’.
> > >>
> > >>
> > >> On Dec 14, 2021, at 10:37 AM, Rui Wang 
> > >> wrote:
> > >>
> > >> Hi community,
> > >>
> > >> To prepare for Calcite 1.29.0 release, I will start to check the
> > >> JIRA
> > >> list
> > >> [1] which marks its fix version as 1.29.0 (9 JIRA as of now remains
> > >> open/in
> > >> progress) and also the pending PRs which looks promising to be
> > >> merged.
> > >> Please help resolve/review/merge JIRA/PRs if you want some fixes to
> > >> be
> > >> released.
> > >>
> > >>
> > >> My current goal is to start freezing the main branch of Calcite by
> > >> the
> > >> end
> > >> of this week.
> > >>
> > >> Please let me know if you have any questions.
> > >>
> > >> [1]:
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> >
> https://issues.apache.org/jira/browse/CALCITE-4941?jql=project%20%3D%20CALCITE%20AND%20status%20in%20(Open%2C%20%22In%20Progress%22)%20AND%20fixVersion%20%3D%201.29.0
> > >>
> > >>
> > >> -Rui
>

Committer permissions

2021-12-18 Thread Vladimir Ozerov
Hi,

It seems that I do not have the committer permissions to the repo, as I can
merge PR neither from GitHub, nor manually. Could you please grant me the
required access rights? Please let me know if any additional information is
required.

Regards,
Vladimir.


  1   2   >