I have logged https://issues.apache.org/jira/browse/CALCITE-4559,
"Create 'interface RexRule', a modular rewrite for row-expressions".
Abstracting RexNode rewrites as objects would be a major step toward
achieving the goals in this thread.
Now is a great chance to give feedback on this design.
Just a quick note so that we are all on the same page.
Currently, the so-called normalization in Calcite does not really perform
any re-ordering of the operands; everything happens inside the
implementation of RexCall#equals.
For example, we are saying that $1 = 'A' is equals to 'A' = $1 by
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
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
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
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
The shuffling is already part of a Hep phase [1] although the same planner
is reused to take advantage of metadata caching etc.
If people feel otherwise then we don't really need to have something super
customisable but just the option to use it or not in a more friendly way
than a system
Julian> If Hive doesn't want its ANDs re-ordered, i
I'm afraid that is a non-starter for Hive use case.
Replacing all ANDs with CASE would defeat simplified and it would defeat
lots of rules.
Julian> * Make normalization less aggressive
It still looks like a customization.
There's one more
> normalization should be customizable.
I'll make the counter-argument. If people can't agree what should be
the "right" normalization, then it isn't normalization, it's
optimization.
A good design doesn't need 100 ways to customize. It needs just a few
powerful ones. We have rewrite rules, and
Stamatis>If downstream projects can pass their own normalizer in the
planner or
Stamatis>possibly in RelBuilder then I guess we can cover pretty much every
use
Stamatis>case.
This is my understanding as well.
Vladimir
I think we agree on this Vladimir, although I don't want to focus too much
on the approach of Hive which I used mostly to have a concrete example but
rather on the fact that normalization should be customizable.
If downstream projects can pass their own normalizer in the planner or
possibly in
Stamatis>a case where digest normalization is not desirable and
Stamatis>brings problems to a downstream project.
It looks more like a custom normalizer to me rather than "normalization is
not desirable".
If I read the problem right, the very same idea was #1 in CALCITE-2450:
<>.
The current
Hello,
Today I bumped into a case where digest normalization is not desirable and
brings problems to a downstream project.
To be more specific, Hive has a rule to reorder operators in a filter
condition for performance reasons [1] which becomes useless after
normalization.
The good news is that
I added and it passed.
Best,
Danny Chan
在 2020年7月16日 +0800 PM4:35,Vladimir Sitnikov ,写道:
> Danny, could you please add a test case for nestes normalization (see
> comment in pr) ?
>
> Vladimir
I would commit the PR soon because I want this into 1.24 release, we can
continue the discussion though ~
Best,
Danny Chan
在 2020年7月16日 +0800 PM2:53,Vladimir Sitnikov ,写道:
> > But currently AND/OR in Calcite is always binary operator
>
> I guess we might want to add multi-arg AND in the future
If the number of operands is greater than a threshold, like 5, or 10, we can
just stop normalizing it.
But currently AND/OR in Calcite is always binary operator, IN is not supported
in RexNode world.
Anyway, the discussion is orthogonal with your pull request, and it doesn't
block the
Well, I think I now got your idea, I agree a specific operator sub-class plus a
special symmetric hash code is more efficient and extensible. But for the first
version, I would only consider binary operators because I could not figure out
how to implement an efficient equals for operator like
> I might be wrong, however, I did see "< vs >" normalization to reduce the
> search space. I added the normalization performance rather than aesthetics
> reasons.
Are you sure the performance gain is caused by "< vs >" normalization instead
of "=" normalization? Can you show me the test case?
>BTW, I don't understand why we need c? Isn't a*17+b good enough to avoid
the corner case?
That depends on the number of corner cases we want to avoid :)
a*17+b might be good enough.
Vladimir
Guava library's approach is pretty naive:
Hashing.combineUnordered
It just adds each byte together, I don't think it is better than the one you
proposed.
BTW, I don't understand why we need c? Isn't a*17+b good enough to avoid the
corner case?
On 2020/07/15 20:33:55, Haisheng Yuan wrote:
>
> a) compute xor of the hashes
> b) compute sum of the hashes
> c) compute product of the hashes (with 0 replaced by 1 to avoid 0*n=0)
> Then combine the three values somehow. For instance: (a*17+b)*17+c
That is really good one. Better than pure XOR.
I also found this in scala:
> things might be bad if we want do dedup RexNode children using
Set
I guess the following might work better than XOR:
a) compute xor of the hashes
b) compute sum of the hashes
c) compute product of the hashes (with 0 replaced by 1 to avoid 0*n=0)
Then combine the three values somehow. For
> 1) I do not like the idea of XOR-based hash code, because it would make
> ($1=$1) have the same hashcode as ($2=$2) and so on.
You have a point. However, is it really a concern? How frequent will it occur?
Especially when an operator like Join, Filter, that has the same input rel, but
with
I agree that extensibility might be helpful, however:
1) I do not like the idea of XOR-based hash code, because it would make
($1=$1) have the same hashcode as ($2=$2) and so on.
2) "$2 > $1 is reordered to $1 < $2, so that predicate a > b and b < a can
be reduced to a > b."
This reverting can
> Customized sql operator can also benefit. [1]
I am not sure if I missed something. Can you show me how can the customized sql
operator benefit from this?
e.g. geospatial operator intersect (it is input order insensitive):
boolean &&( geometry A , geometry B )
> Add a SqlOperator interface is
I have extended the logic to support all the symmetrical operators(=, <> ..)
and the binary comparison operators (>=, < ..),
not only just RexInputRef. Customized sql operator can also benefit. [1]
The way is to compare the operands with SqlKind first then fallback to their
hashcode (eargely
+1 to move rex node normalization out of constructor.
On a second thought, I can't help thinking, do we really need RexNode
normalization?
There are 2 kinds of normalization in current codebase:
1. reverse the operator
$2 > $1 is reordered to $1 < $2, so that predicate a > b and b < a can be
Yes, it is. We can keep it as a builtin promotion.
Best,
Danny Chan
在 2020年7月13日 +0800 PM3:48,Vladimir Sitnikov ,写道:
> > Hi, all, I’m planning to default disable the RexNode normalization in
> CALCITE-4073, if you have any objections, please let me know in 24 hours,
> thanks so much ~
>
> I
>Hi, all, I’m planning to default disable the RexNode normalization in
CALCITE-4073, if you have any objections, please let me know in 24 hours,
thanks so much ~
I assume it would still normalize RexNodes when building plan digest. Is it
the case?
Vladimir
Hi, all, I’m planning to default disable the RexNode normalization in
CALCITE-4073, if you have any objections, please let me know in 24 hours,
thanks so much ~
Looking forward to your feedback ~
Best,
Danny Chan
在 2020年7月8日 +0800 PM4:38,Danny Chan ,写道:
> In CALCITE-2450, we proposed a change
I mean default to not make normalization. This feature starts from release
1.22.0.
Stamatis Zampetakis 于2020年7月11日 周六上午7:04写道:
> Hi Danny,
>
> From the initial discussion, I was leaning more towards the idea of having
> a separate component (something like what is described in CALCITE-4073)
>
Hi Danny,
>From the initial discussion, I was leaning more towards the idea of having
a separate component (something like what is described in CALCITE-4073)
that people can use if they really need to.
>From my experience so far I can confirm that different projects have
different needs in terms
In CALCITE-2450, we proposed a change to normalize the RexNode, and there is a
discussion[1], the change is in very early phrase and the normalization
pattern is unstable.
There is actually no common consensus about what a form (or pattern) a desired
normalization should be:
• People may
33 matches
Mail list logo