You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@arrow.apache.org by Ben Kietzman <be...@rstudio.com> on 2020/06/03 14:47:38 UTC

[DISCUSS] Add kernel integer overflow handling

https://github.com/apache/arrow/pull/7341#issuecomment-638241193

How should arithmetic kernels handle integer overflow?

The approach currently taken in the linked PR is to promote such that
overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
uint16)->uint32`.

I'm not sure that's desirable. For one thing this leads to inconsistent
handling of 64 bit integer types, which are currently allowed to overflow
since we cannot promote further (NB: that means this kernel includes
undefined behavior for int64).

There are a few other approaches we could take (ordered by personal
preference):

   - define explicit overflow behavior for signed integer operands (for
   example if we declared that add(i8(a), i8(b)) will always be equivalent
   to i8(i16(a) + i16(b)) then we could instantiate only unsigned addition
   kernels)
   - raise an error on signed overflow
   - provide ArithmeticOptions::overflow_behavior and allow users to choose
   between these
   - require users to pass arguments which will not overflow

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Wes McKinney <we...@gmail.com>.
On Wed, Jun 3, 2020 at 10:44 AM Wes McKinney <we...@gmail.com> wrote:
>
> > By default an error should probably be raised
>
> I would very strongly recommend keeping the behavior consistent with
> that of analytic DBMSes. I don't think that most analytic DBMS error
> on overflows because it's too computationally expensive to check.
> NumPy doesn't error (by default at least) when you have overflows
> either:
>
> >>> arr = np.array([128, 128, 128], dtype='i1')
> >>> arr + arr
> array([0, 0, 0], dtype=int8)

*facepalm*

>>> arr = np.array([127, 127, 127], dtype='i1')
>>> arr + arr
array([-2, -2, -2], dtype=int8)

>>>


>
> On Wed, Jun 3, 2020 at 10:29 AM Antoine Pitrou <so...@pitrou.net> wrote:
> >
> > On Wed, 3 Jun 2020 10:47:38 -0400
> > Ben Kietzman <be...@rstudio.com> wrote:
> > > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > >
> > > How should arithmetic kernels handle integer overflow?
> > >
> > > The approach currently taken in the linked PR is to promote such that
> > > overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> > > uint16)->uint32`.
> > >
> > > I'm not sure that's desirable. For one thing this leads to inconsistent
> > > handling of 64 bit integer types, which are currently allowed to overflow
> > > since we cannot promote further (NB: that means this kernel includes
> > > undefined behavior for int64).
> >
> > I agree with you.  I would strongly advise against implicit promotion
> > accross arithmetic operations.  We initially did that in Numba and it
> > quickly became a can of worms.
> >
> > The most desirable behaviour IMHO is to keep the original type, so:
> > - (int8, int8) -> int8
> > - (uint16, uint16) -> uint16
> >
> > Then the question is what happens when the actual overflow occurs.  I
> > think this should be directed by a kernel option.  By default an error
> > should probably be raised (letting errors pass and silently produce
> > erroneous data is wrong), but we might want to allow people to bypass
> > overflow checks for speed.
> >
> > Note that even if overflow detection is enabled, it *should* be possible
> > to enable vectorization, e.g. by making overflow detection a separate
> > pass (itself vectorizable).
> >
> > Regards
> >
> > Antoine.
> >
> >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Wes McKinney <we...@gmail.com>.
> By default an error should probably be raised

I would very strongly recommend keeping the behavior consistent with
that of analytic DBMSes. I don't think that most analytic DBMS error
on overflows because it's too computationally expensive to check.
NumPy doesn't error (by default at least) when you have overflows
either:

>>> arr = np.array([128, 128, 128], dtype='i1')
>>> arr + arr
array([0, 0, 0], dtype=int8)

On Wed, Jun 3, 2020 at 10:29 AM Antoine Pitrou <so...@pitrou.net> wrote:
>
> On Wed, 3 Jun 2020 10:47:38 -0400
> Ben Kietzman <be...@rstudio.com> wrote:
> > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> >
> > How should arithmetic kernels handle integer overflow?
> >
> > The approach currently taken in the linked PR is to promote such that
> > overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> > uint16)->uint32`.
> >
> > I'm not sure that's desirable. For one thing this leads to inconsistent
> > handling of 64 bit integer types, which are currently allowed to overflow
> > since we cannot promote further (NB: that means this kernel includes
> > undefined behavior for int64).
>
> I agree with you.  I would strongly advise against implicit promotion
> accross arithmetic operations.  We initially did that in Numba and it
> quickly became a can of worms.
>
> The most desirable behaviour IMHO is to keep the original type, so:
> - (int8, int8) -> int8
> - (uint16, uint16) -> uint16
>
> Then the question is what happens when the actual overflow occurs.  I
> think this should be directed by a kernel option.  By default an error
> should probably be raised (letting errors pass and silently produce
> erroneous data is wrong), but we might want to allow people to bypass
> overflow checks for speed.
>
> Note that even if overflow detection is enabled, it *should* be possible
> to enable vectorization, e.g. by making overflow detection a separate
> pass (itself vectorizable).
>
> Regards
>
> Antoine.
>
>

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Wes McKinney <we...@gmail.com>.
On Wed, Jun 3, 2020 at 11:16 AM Krisztián Szűcs
<sz...@gmail.com> wrote:
>
> On Wed, Jun 3, 2020 at 5:52 PM Wes McKinney <we...@gmail.com> wrote:
> >
> > On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
> > <sz...@gmail.com> wrote:
> > >
> > > From the user perspective I find the following pretty confusing:
> > >
> > > In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> > > Out[1]: array([ 0, -2], dtype=int8)
> > >
> > > In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> > > Out[2]: array([-256,  254], dtype=int16)
> > >
> > > In my opinion somewhere (on a higher level maybe) we should provide
> > > the correct results promoted to a wider type implicitly.
> >
> > Yes, I agree with you, but I agree that the best place to address this
> > is at a higher level rather than having this logic implemented at the
> > lowest level (kernels) -- I think database systems handle this during
> > logical->physical planning.
>
> It raises another question: where to incorporate the implicit promotions?
> // correct me if I'm wrong but these implicit promotions are operation
> // dependent and distinct from kernel dispatching issue [1]

Implicit promotions would be handled when translating a logical
expression (like "Add(x, y)") to a physical bound expression
(something like "add_kernel[int16, int16](cast(x, int16), cast(y,
int16))", where x and y are int8).

>
> The numpy example above can be roughly translated to:
> >>> a = pa.array([-128, 127])
> >>> pa.compute.add(a, a)
> array([ 0, -2]
>
> Which is rather surprising from the user's perspective.

I guess we have to qualify what is a "user". The direct "user" of the
kernels is not the same as a user who would write a SQL query or use a
data frame library. I believe the contents of pyarrow.compute (and
most of pyarrow, FWIW) are intended for system developers not end
users (e.g. "people who use pandas").

>
> [1] https://issues.apache.org/jira/browse/ARROW-8919
> >
> > > Clickhouse for example does the type promotion.
> > >
> > > On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net> wrote:
> > > >
> > > > On Wed, 3 Jun 2020 10:47:38 -0400
> > > > Ben Kietzman <be...@rstudio.com> wrote:
> > > > > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > > > >
> > > > > How should arithmetic kernels handle integer overflow?
> > > > >
> > > > > The approach currently taken in the linked PR is to promote such that
> > > > > overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> > > > > uint16)->uint32`.
> > > > >
> > > > > I'm not sure that's desirable. For one thing this leads to inconsistent
> > > > > handling of 64 bit integer types, which are currently allowed to overflow
> > > > > since we cannot promote further (NB: that means this kernel includes
> > > > > undefined behavior for int64).
> > > >
> > > > I agree with you.  I would strongly advise against implicit promotion
> > > > accross arithmetic operations.  We initially did that in Numba and it
> > > > quickly became a can of worms.
> > > >
> > > > The most desirable behaviour IMHO is to keep the original type, so:
> > > > - (int8, int8) -> int8
> > > > - (uint16, uint16) -> uint16
> > > >
> > > > Then the question is what happens when the actual overflow occurs.  I
> > > > think this should be directed by a kernel option.  By default an error
> > > > should probably be raised (letting errors pass and silently produce
> > > > erroneous data is wrong), but we might want to allow people to bypass
> > > > overflow checks for speed.
> > > >
> > > > Note that even if overflow detection is enabled, it *should* be possible
> > > > to enable vectorization, e.g. by making overflow detection a separate
> > > > pass (itself vectorizable).
> > > >
> > > > Regards
> > > >
> > > > Antoine.
> > > >
> > > >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Francois Saint-Jacques <fs...@gmail.com>.
I documented [1] the behaviors by experimentation or by reading the
documentation. My experiments were mostly about checking INT64_MAX +
1. My preference would be to use the platform defined behavior by
default and provide a safety option that errors.

Feel  free to add more databases/systems.

François

[1] https://docs.google.com/spreadsheets/d/1DTFER2arYyNkjEHd1jCLdhQ90kRHbNrVI-UvHvQiP6Y/edit?usp=sharing

On Thu, Jun 4, 2020 at 9:22 AM Wes McKinney <we...@gmail.com> wrote:
>
> On Thu, Jun 4, 2020 at 4:57 AM Krisztián Szűcs
> <sz...@gmail.com> wrote:
> >
> > On Thu, Jun 4, 2020 at 11:09 AM Rémi Dettai <rd...@gmail.com> wrote:
> > >
> > > It makes sense to me that the default behaviour of such a low level api as
> > > kernel does not do any automagic promotion, but shouldn't this kind of
> > > promotion still be requestable by the so called "system developer" user ?
> > > Otherwise he would need to materialize a promoted version of each original
> > > array before the kernel operation, wouldn't he ?
>
> This will all be handled by the expression execution system. Please, I
> beg your collective patience as I will be developing this in the near
> future.
>
> > I assume yes, for 1 million elements:
> >
> > Promotion inside kernel:
> > u32 + u32 = u64
> > 4MB + 4MB -> 8MB
> > New allocation: 8MB
> >
> > Promotion outside kernel:
> > (u32 -> u64) + (u32 -> u64) = u64
> > (4MB -> 8MB) + (4MB -> 8MB) -> 8MB
> > New allocation: 24MB
>
> This isn't what the execution engine will do, though. The processed
> chunks will be much smaller, like 64K elements or less, so at no point
> will such large temporary allocations exist.
>
> > >
> > > Le mer. 3 juin 2020 à 18:27, Wes McKinney <we...@gmail.com> a écrit :
> > >
> > > > On Wed, Jun 3, 2020 at 11:25 AM Krisztián Szűcs
> > > > <sz...@gmail.com> wrote:
> > > > >
> > > > > On Wed, Jun 3, 2020 at 6:16 PM Krisztián Szűcs
> > > > > <sz...@gmail.com> wrote:
> > > > > >
> > > > > > On Wed, Jun 3, 2020 at 5:52 PM Wes McKinney <we...@gmail.com>
> > > > wrote:
> > > > > > >
> > > > > > > On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
> > > > > > > <sz...@gmail.com> wrote:
> > > > > > > >
> > > > > > > > From the user perspective I find the following pretty confusing:
> > > > > > > >
> > > > > > > > In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> > > > > > > > Out[1]: array([ 0, -2], dtype=int8)
> > > > > > > >
> > > > > > > > In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> > > > > > > > Out[2]: array([-256,  254], dtype=int16)
> > > > > > > >
> > > > > > > > In my opinion somewhere (on a higher level maybe) we should provide
> > > > > > > > the correct results promoted to a wider type implicitly.
> > > > > > >
> > > > > > > Yes, I agree with you, but I agree that the best place to address
> > > > this
> > > > > > > is at a higher level rather than having this logic implemented at the
> > > > > > > lowest level (kernels) -- I think database systems handle this during
> > > > > > > logical->physical planning.
> > > > > >
> > > > > > It raises another question: where to incorporate the implicit
> > > > promotions?
> > > > > > // correct me if I'm wrong but these implicit promotions are operation
> > > > > > // dependent and distinct from kernel dispatching issue [1]
> > > > > >
> > > > > > The numpy example above can be roughly translated to:
> > > > > > >>> a = pa.array([-128, 127])
> > > > > > >>> pa.compute.add(a, a)
> > > > > > array([ 0, -2]
> > > > > >
> > > > > > Which is rather surprising from the user's perspective.
> > > > >
> > > > > Would it be enough to document the exact behavior  and advice the user
> > > > > to place casts until we have logical -> phisycal machinery?
> > > >
> > > > I think it's enough to clearly document the behavior and assume that
> > > > the "user" will act according to what semantics are desired for their
> > > > use cases. Per my comments in my last e-mail I don't think the users
> > > > of these functions need to be handled with "kid's gloves".
> > > >
> > > > > I'm updating my PR as discussed.
> > > > > >
> > > > > > [1] https://issues.apache.org/jira/browse/ARROW-8919
> > > > > > >
> > > > > > > > Clickhouse for example does the type promotion.
> > > > > > > >
> > > > > > > > On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > On Wed, 3 Jun 2020 10:47:38 -0400
> > > > > > > > > Ben Kietzman <be...@rstudio.com> wrote:
> > > > > > > > > >
> > > > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > > > > > > > > >
> > > > > > > > > > How should arithmetic kernels handle integer overflow?
> > > > > > > > > >
> > > > > > > > > > The approach currently taken in the linked PR is to promote
> > > > such that
> > > > > > > > > > overflow will not occur, for example `(int8, int8)->int16` and
> > > > `(uint16,
> > > > > > > > > > uint16)->uint32`.
> > > > > > > > > >
> > > > > > > > > > I'm not sure that's desirable. For one thing this leads to
> > > > inconsistent
> > > > > > > > > > handling of 64 bit integer types, which are currently allowed
> > > > to overflow
> > > > > > > > > > since we cannot promote further (NB: that means this kernel
> > > > includes
> > > > > > > > > > undefined behavior for int64).
> > > > > > > > >
> > > > > > > > > I agree with you.  I would strongly advise against implicit
> > > > promotion
> > > > > > > > > accross arithmetic operations.  We initially did that in Numba
> > > > and it
> > > > > > > > > quickly became a can of worms.
> > > > > > > > >
> > > > > > > > > The most desirable behaviour IMHO is to keep the original type,
> > > > so:
> > > > > > > > > - (int8, int8) -> int8
> > > > > > > > > - (uint16, uint16) -> uint16
> > > > > > > > >
> > > > > > > > > Then the question is what happens when the actual overflow
> > > > occurs.  I
> > > > > > > > > think this should be directed by a kernel option.  By default an
> > > > error
> > > > > > > > > should probably be raised (letting errors pass and silently
> > > > produce
> > > > > > > > > erroneous data is wrong), but we might want to allow people to
> > > > bypass
> > > > > > > > > overflow checks for speed.
> > > > > > > > >
> > > > > > > > > Note that even if overflow detection is enabled, it *should* be
> > > > possible
> > > > > > > > > to enable vectorization, e.g. by making overflow detection a
> > > > separate
> > > > > > > > > pass (itself vectorizable).
> > > > > > > > >
> > > > > > > > > Regards
> > > > > > > > >
> > > > > > > > > Antoine.
> > > > > > > > >
> > > > > > > > >
> > > >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Wes McKinney <we...@gmail.com>.
On Thu, Jun 4, 2020 at 4:57 AM Krisztián Szűcs
<sz...@gmail.com> wrote:
>
> On Thu, Jun 4, 2020 at 11:09 AM Rémi Dettai <rd...@gmail.com> wrote:
> >
> > It makes sense to me that the default behaviour of such a low level api as
> > kernel does not do any automagic promotion, but shouldn't this kind of
> > promotion still be requestable by the so called "system developer" user ?
> > Otherwise he would need to materialize a promoted version of each original
> > array before the kernel operation, wouldn't he ?

This will all be handled by the expression execution system. Please, I
beg your collective patience as I will be developing this in the near
future.

> I assume yes, for 1 million elements:
>
> Promotion inside kernel:
> u32 + u32 = u64
> 4MB + 4MB -> 8MB
> New allocation: 8MB
>
> Promotion outside kernel:
> (u32 -> u64) + (u32 -> u64) = u64
> (4MB -> 8MB) + (4MB -> 8MB) -> 8MB
> New allocation: 24MB

This isn't what the execution engine will do, though. The processed
chunks will be much smaller, like 64K elements or less, so at no point
will such large temporary allocations exist.

> >
> > Le mer. 3 juin 2020 à 18:27, Wes McKinney <we...@gmail.com> a écrit :
> >
> > > On Wed, Jun 3, 2020 at 11:25 AM Krisztián Szűcs
> > > <sz...@gmail.com> wrote:
> > > >
> > > > On Wed, Jun 3, 2020 at 6:16 PM Krisztián Szűcs
> > > > <sz...@gmail.com> wrote:
> > > > >
> > > > > On Wed, Jun 3, 2020 at 5:52 PM Wes McKinney <we...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
> > > > > > <sz...@gmail.com> wrote:
> > > > > > >
> > > > > > > From the user perspective I find the following pretty confusing:
> > > > > > >
> > > > > > > In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> > > > > > > Out[1]: array([ 0, -2], dtype=int8)
> > > > > > >
> > > > > > > In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> > > > > > > Out[2]: array([-256,  254], dtype=int16)
> > > > > > >
> > > > > > > In my opinion somewhere (on a higher level maybe) we should provide
> > > > > > > the correct results promoted to a wider type implicitly.
> > > > > >
> > > > > > Yes, I agree with you, but I agree that the best place to address
> > > this
> > > > > > is at a higher level rather than having this logic implemented at the
> > > > > > lowest level (kernels) -- I think database systems handle this during
> > > > > > logical->physical planning.
> > > > >
> > > > > It raises another question: where to incorporate the implicit
> > > promotions?
> > > > > // correct me if I'm wrong but these implicit promotions are operation
> > > > > // dependent and distinct from kernel dispatching issue [1]
> > > > >
> > > > > The numpy example above can be roughly translated to:
> > > > > >>> a = pa.array([-128, 127])
> > > > > >>> pa.compute.add(a, a)
> > > > > array([ 0, -2]
> > > > >
> > > > > Which is rather surprising from the user's perspective.
> > > >
> > > > Would it be enough to document the exact behavior  and advice the user
> > > > to place casts until we have logical -> phisycal machinery?
> > >
> > > I think it's enough to clearly document the behavior and assume that
> > > the "user" will act according to what semantics are desired for their
> > > use cases. Per my comments in my last e-mail I don't think the users
> > > of these functions need to be handled with "kid's gloves".
> > >
> > > > I'm updating my PR as discussed.
> > > > >
> > > > > [1] https://issues.apache.org/jira/browse/ARROW-8919
> > > > > >
> > > > > > > Clickhouse for example does the type promotion.
> > > > > > >
> > > > > > > On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net>
> > > wrote:
> > > > > > > >
> > > > > > > > On Wed, 3 Jun 2020 10:47:38 -0400
> > > > > > > > Ben Kietzman <be...@rstudio.com> wrote:
> > > > > > > > >
> > > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > > > > > > > >
> > > > > > > > > How should arithmetic kernels handle integer overflow?
> > > > > > > > >
> > > > > > > > > The approach currently taken in the linked PR is to promote
> > > such that
> > > > > > > > > overflow will not occur, for example `(int8, int8)->int16` and
> > > `(uint16,
> > > > > > > > > uint16)->uint32`.
> > > > > > > > >
> > > > > > > > > I'm not sure that's desirable. For one thing this leads to
> > > inconsistent
> > > > > > > > > handling of 64 bit integer types, which are currently allowed
> > > to overflow
> > > > > > > > > since we cannot promote further (NB: that means this kernel
> > > includes
> > > > > > > > > undefined behavior for int64).
> > > > > > > >
> > > > > > > > I agree with you.  I would strongly advise against implicit
> > > promotion
> > > > > > > > accross arithmetic operations.  We initially did that in Numba
> > > and it
> > > > > > > > quickly became a can of worms.
> > > > > > > >
> > > > > > > > The most desirable behaviour IMHO is to keep the original type,
> > > so:
> > > > > > > > - (int8, int8) -> int8
> > > > > > > > - (uint16, uint16) -> uint16
> > > > > > > >
> > > > > > > > Then the question is what happens when the actual overflow
> > > occurs.  I
> > > > > > > > think this should be directed by a kernel option.  By default an
> > > error
> > > > > > > > should probably be raised (letting errors pass and silently
> > > produce
> > > > > > > > erroneous data is wrong), but we might want to allow people to
> > > bypass
> > > > > > > > overflow checks for speed.
> > > > > > > >
> > > > > > > > Note that even if overflow detection is enabled, it *should* be
> > > possible
> > > > > > > > to enable vectorization, e.g. by making overflow detection a
> > > separate
> > > > > > > > pass (itself vectorizable).
> > > > > > > >
> > > > > > > > Regards
> > > > > > > >
> > > > > > > > Antoine.
> > > > > > > >
> > > > > > > >
> > >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Krisztián Szűcs <sz...@gmail.com>.
On Thu, Jun 4, 2020 at 11:09 AM Rémi Dettai <rd...@gmail.com> wrote:
>
> It makes sense to me that the default behaviour of such a low level api as
> kernel does not do any automagic promotion, but shouldn't this kind of
> promotion still be requestable by the so called "system developer" user ?
> Otherwise he would need to materialize a promoted version of each original
> array before the kernel operation, wouldn't he ?

I assume yes, for 1 million elements:

Promotion inside kernel:
u32 + u32 = u64
4MB + 4MB -> 8MB
New allocation: 8MB

Promotion outside kernel:
(u32 -> u64) + (u32 -> u64) = u64
(4MB -> 8MB) + (4MB -> 8MB) -> 8MB
New allocation: 24MB

>
> Le mer. 3 juin 2020 à 18:27, Wes McKinney <we...@gmail.com> a écrit :
>
> > On Wed, Jun 3, 2020 at 11:25 AM Krisztián Szűcs
> > <sz...@gmail.com> wrote:
> > >
> > > On Wed, Jun 3, 2020 at 6:16 PM Krisztián Szűcs
> > > <sz...@gmail.com> wrote:
> > > >
> > > > On Wed, Jun 3, 2020 at 5:52 PM Wes McKinney <we...@gmail.com>
> > wrote:
> > > > >
> > > > > On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
> > > > > <sz...@gmail.com> wrote:
> > > > > >
> > > > > > From the user perspective I find the following pretty confusing:
> > > > > >
> > > > > > In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> > > > > > Out[1]: array([ 0, -2], dtype=int8)
> > > > > >
> > > > > > In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> > > > > > Out[2]: array([-256,  254], dtype=int16)
> > > > > >
> > > > > > In my opinion somewhere (on a higher level maybe) we should provide
> > > > > > the correct results promoted to a wider type implicitly.
> > > > >
> > > > > Yes, I agree with you, but I agree that the best place to address
> > this
> > > > > is at a higher level rather than having this logic implemented at the
> > > > > lowest level (kernels) -- I think database systems handle this during
> > > > > logical->physical planning.
> > > >
> > > > It raises another question: where to incorporate the implicit
> > promotions?
> > > > // correct me if I'm wrong but these implicit promotions are operation
> > > > // dependent and distinct from kernel dispatching issue [1]
> > > >
> > > > The numpy example above can be roughly translated to:
> > > > >>> a = pa.array([-128, 127])
> > > > >>> pa.compute.add(a, a)
> > > > array([ 0, -2]
> > > >
> > > > Which is rather surprising from the user's perspective.
> > >
> > > Would it be enough to document the exact behavior  and advice the user
> > > to place casts until we have logical -> phisycal machinery?
> >
> > I think it's enough to clearly document the behavior and assume that
> > the "user" will act according to what semantics are desired for their
> > use cases. Per my comments in my last e-mail I don't think the users
> > of these functions need to be handled with "kid's gloves".
> >
> > > I'm updating my PR as discussed.
> > > >
> > > > [1] https://issues.apache.org/jira/browse/ARROW-8919
> > > > >
> > > > > > Clickhouse for example does the type promotion.
> > > > > >
> > > > > > On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net>
> > wrote:
> > > > > > >
> > > > > > > On Wed, 3 Jun 2020 10:47:38 -0400
> > > > > > > Ben Kietzman <be...@rstudio.com> wrote:
> > > > > > > >
> > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > > > > > > >
> > > > > > > > How should arithmetic kernels handle integer overflow?
> > > > > > > >
> > > > > > > > The approach currently taken in the linked PR is to promote
> > such that
> > > > > > > > overflow will not occur, for example `(int8, int8)->int16` and
> > `(uint16,
> > > > > > > > uint16)->uint32`.
> > > > > > > >
> > > > > > > > I'm not sure that's desirable. For one thing this leads to
> > inconsistent
> > > > > > > > handling of 64 bit integer types, which are currently allowed
> > to overflow
> > > > > > > > since we cannot promote further (NB: that means this kernel
> > includes
> > > > > > > > undefined behavior for int64).
> > > > > > >
> > > > > > > I agree with you.  I would strongly advise against implicit
> > promotion
> > > > > > > accross arithmetic operations.  We initially did that in Numba
> > and it
> > > > > > > quickly became a can of worms.
> > > > > > >
> > > > > > > The most desirable behaviour IMHO is to keep the original type,
> > so:
> > > > > > > - (int8, int8) -> int8
> > > > > > > - (uint16, uint16) -> uint16
> > > > > > >
> > > > > > > Then the question is what happens when the actual overflow
> > occurs.  I
> > > > > > > think this should be directed by a kernel option.  By default an
> > error
> > > > > > > should probably be raised (letting errors pass and silently
> > produce
> > > > > > > erroneous data is wrong), but we might want to allow people to
> > bypass
> > > > > > > overflow checks for speed.
> > > > > > >
> > > > > > > Note that even if overflow detection is enabled, it *should* be
> > possible
> > > > > > > to enable vectorization, e.g. by making overflow detection a
> > separate
> > > > > > > pass (itself vectorizable).
> > > > > > >
> > > > > > > Regards
> > > > > > >
> > > > > > > Antoine.
> > > > > > >
> > > > > > >
> >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Rémi Dettai <rd...@gmail.com>.
It makes sense to me that the default behaviour of such a low level api as
kernel does not do any automagic promotion, but shouldn't this kind of
promotion still be requestable by the so called "system developer" user ?
Otherwise he would need to materialize a promoted version of each original
array before the kernel operation, wouldn't he ?

Le mer. 3 juin 2020 à 18:27, Wes McKinney <we...@gmail.com> a écrit :

> On Wed, Jun 3, 2020 at 11:25 AM Krisztián Szűcs
> <sz...@gmail.com> wrote:
> >
> > On Wed, Jun 3, 2020 at 6:16 PM Krisztián Szűcs
> > <sz...@gmail.com> wrote:
> > >
> > > On Wed, Jun 3, 2020 at 5:52 PM Wes McKinney <we...@gmail.com>
> wrote:
> > > >
> > > > On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
> > > > <sz...@gmail.com> wrote:
> > > > >
> > > > > From the user perspective I find the following pretty confusing:
> > > > >
> > > > > In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> > > > > Out[1]: array([ 0, -2], dtype=int8)
> > > > >
> > > > > In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> > > > > Out[2]: array([-256,  254], dtype=int16)
> > > > >
> > > > > In my opinion somewhere (on a higher level maybe) we should provide
> > > > > the correct results promoted to a wider type implicitly.
> > > >
> > > > Yes, I agree with you, but I agree that the best place to address
> this
> > > > is at a higher level rather than having this logic implemented at the
> > > > lowest level (kernels) -- I think database systems handle this during
> > > > logical->physical planning.
> > >
> > > It raises another question: where to incorporate the implicit
> promotions?
> > > // correct me if I'm wrong but these implicit promotions are operation
> > > // dependent and distinct from kernel dispatching issue [1]
> > >
> > > The numpy example above can be roughly translated to:
> > > >>> a = pa.array([-128, 127])
> > > >>> pa.compute.add(a, a)
> > > array([ 0, -2]
> > >
> > > Which is rather surprising from the user's perspective.
> >
> > Would it be enough to document the exact behavior  and advice the user
> > to place casts until we have logical -> phisycal machinery?
>
> I think it's enough to clearly document the behavior and assume that
> the "user" will act according to what semantics are desired for their
> use cases. Per my comments in my last e-mail I don't think the users
> of these functions need to be handled with "kid's gloves".
>
> > I'm updating my PR as discussed.
> > >
> > > [1] https://issues.apache.org/jira/browse/ARROW-8919
> > > >
> > > > > Clickhouse for example does the type promotion.
> > > > >
> > > > > On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net>
> wrote:
> > > > > >
> > > > > > On Wed, 3 Jun 2020 10:47:38 -0400
> > > > > > Ben Kietzman <be...@rstudio.com> wrote:
> > > > > > >
> https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > > > > > >
> > > > > > > How should arithmetic kernels handle integer overflow?
> > > > > > >
> > > > > > > The approach currently taken in the linked PR is to promote
> such that
> > > > > > > overflow will not occur, for example `(int8, int8)->int16` and
> `(uint16,
> > > > > > > uint16)->uint32`.
> > > > > > >
> > > > > > > I'm not sure that's desirable. For one thing this leads to
> inconsistent
> > > > > > > handling of 64 bit integer types, which are currently allowed
> to overflow
> > > > > > > since we cannot promote further (NB: that means this kernel
> includes
> > > > > > > undefined behavior for int64).
> > > > > >
> > > > > > I agree with you.  I would strongly advise against implicit
> promotion
> > > > > > accross arithmetic operations.  We initially did that in Numba
> and it
> > > > > > quickly became a can of worms.
> > > > > >
> > > > > > The most desirable behaviour IMHO is to keep the original type,
> so:
> > > > > > - (int8, int8) -> int8
> > > > > > - (uint16, uint16) -> uint16
> > > > > >
> > > > > > Then the question is what happens when the actual overflow
> occurs.  I
> > > > > > think this should be directed by a kernel option.  By default an
> error
> > > > > > should probably be raised (letting errors pass and silently
> produce
> > > > > > erroneous data is wrong), but we might want to allow people to
> bypass
> > > > > > overflow checks for speed.
> > > > > >
> > > > > > Note that even if overflow detection is enabled, it *should* be
> possible
> > > > > > to enable vectorization, e.g. by making overflow detection a
> separate
> > > > > > pass (itself vectorizable).
> > > > > >
> > > > > > Regards
> > > > > >
> > > > > > Antoine.
> > > > > >
> > > > > >
>

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Wes McKinney <we...@gmail.com>.
On Wed, Jun 3, 2020 at 11:25 AM Krisztián Szűcs
<sz...@gmail.com> wrote:
>
> On Wed, Jun 3, 2020 at 6:16 PM Krisztián Szűcs
> <sz...@gmail.com> wrote:
> >
> > On Wed, Jun 3, 2020 at 5:52 PM Wes McKinney <we...@gmail.com> wrote:
> > >
> > > On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
> > > <sz...@gmail.com> wrote:
> > > >
> > > > From the user perspective I find the following pretty confusing:
> > > >
> > > > In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> > > > Out[1]: array([ 0, -2], dtype=int8)
> > > >
> > > > In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> > > > Out[2]: array([-256,  254], dtype=int16)
> > > >
> > > > In my opinion somewhere (on a higher level maybe) we should provide
> > > > the correct results promoted to a wider type implicitly.
> > >
> > > Yes, I agree with you, but I agree that the best place to address this
> > > is at a higher level rather than having this logic implemented at the
> > > lowest level (kernels) -- I think database systems handle this during
> > > logical->physical planning.
> >
> > It raises another question: where to incorporate the implicit promotions?
> > // correct me if I'm wrong but these implicit promotions are operation
> > // dependent and distinct from kernel dispatching issue [1]
> >
> > The numpy example above can be roughly translated to:
> > >>> a = pa.array([-128, 127])
> > >>> pa.compute.add(a, a)
> > array([ 0, -2]
> >
> > Which is rather surprising from the user's perspective.
>
> Would it be enough to document the exact behavior  and advice the user
> to place casts until we have logical -> phisycal machinery?

I think it's enough to clearly document the behavior and assume that
the "user" will act according to what semantics are desired for their
use cases. Per my comments in my last e-mail I don't think the users
of these functions need to be handled with "kid's gloves".

> I'm updating my PR as discussed.
> >
> > [1] https://issues.apache.org/jira/browse/ARROW-8919
> > >
> > > > Clickhouse for example does the type promotion.
> > > >
> > > > On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net> wrote:
> > > > >
> > > > > On Wed, 3 Jun 2020 10:47:38 -0400
> > > > > Ben Kietzman <be...@rstudio.com> wrote:
> > > > > > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > > > > >
> > > > > > How should arithmetic kernels handle integer overflow?
> > > > > >
> > > > > > The approach currently taken in the linked PR is to promote such that
> > > > > > overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> > > > > > uint16)->uint32`.
> > > > > >
> > > > > > I'm not sure that's desirable. For one thing this leads to inconsistent
> > > > > > handling of 64 bit integer types, which are currently allowed to overflow
> > > > > > since we cannot promote further (NB: that means this kernel includes
> > > > > > undefined behavior for int64).
> > > > >
> > > > > I agree with you.  I would strongly advise against implicit promotion
> > > > > accross arithmetic operations.  We initially did that in Numba and it
> > > > > quickly became a can of worms.
> > > > >
> > > > > The most desirable behaviour IMHO is to keep the original type, so:
> > > > > - (int8, int8) -> int8
> > > > > - (uint16, uint16) -> uint16
> > > > >
> > > > > Then the question is what happens when the actual overflow occurs.  I
> > > > > think this should be directed by a kernel option.  By default an error
> > > > > should probably be raised (letting errors pass and silently produce
> > > > > erroneous data is wrong), but we might want to allow people to bypass
> > > > > overflow checks for speed.
> > > > >
> > > > > Note that even if overflow detection is enabled, it *should* be possible
> > > > > to enable vectorization, e.g. by making overflow detection a separate
> > > > > pass (itself vectorizable).
> > > > >
> > > > > Regards
> > > > >
> > > > > Antoine.
> > > > >
> > > > >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Krisztián Szűcs <sz...@gmail.com>.
On Wed, Jun 3, 2020 at 6:16 PM Krisztián Szűcs
<sz...@gmail.com> wrote:
>
> On Wed, Jun 3, 2020 at 5:52 PM Wes McKinney <we...@gmail.com> wrote:
> >
> > On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
> > <sz...@gmail.com> wrote:
> > >
> > > From the user perspective I find the following pretty confusing:
> > >
> > > In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> > > Out[1]: array([ 0, -2], dtype=int8)
> > >
> > > In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> > > Out[2]: array([-256,  254], dtype=int16)
> > >
> > > In my opinion somewhere (on a higher level maybe) we should provide
> > > the correct results promoted to a wider type implicitly.
> >
> > Yes, I agree with you, but I agree that the best place to address this
> > is at a higher level rather than having this logic implemented at the
> > lowest level (kernels) -- I think database systems handle this during
> > logical->physical planning.
>
> It raises another question: where to incorporate the implicit promotions?
> // correct me if I'm wrong but these implicit promotions are operation
> // dependent and distinct from kernel dispatching issue [1]
>
> The numpy example above can be roughly translated to:
> >>> a = pa.array([-128, 127])
> >>> pa.compute.add(a, a)
> array([ 0, -2]
>
> Which is rather surprising from the user's perspective.

Would it be enough to document the exact behavior  and advice the user
to place casts until we have logical -> phisycal machinery?

I'm updating my PR as discussed.
>
> [1] https://issues.apache.org/jira/browse/ARROW-8919
> >
> > > Clickhouse for example does the type promotion.
> > >
> > > On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net> wrote:
> > > >
> > > > On Wed, 3 Jun 2020 10:47:38 -0400
> > > > Ben Kietzman <be...@rstudio.com> wrote:
> > > > > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > > > >
> > > > > How should arithmetic kernels handle integer overflow?
> > > > >
> > > > > The approach currently taken in the linked PR is to promote such that
> > > > > overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> > > > > uint16)->uint32`.
> > > > >
> > > > > I'm not sure that's desirable. For one thing this leads to inconsistent
> > > > > handling of 64 bit integer types, which are currently allowed to overflow
> > > > > since we cannot promote further (NB: that means this kernel includes
> > > > > undefined behavior for int64).
> > > >
> > > > I agree with you.  I would strongly advise against implicit promotion
> > > > accross arithmetic operations.  We initially did that in Numba and it
> > > > quickly became a can of worms.
> > > >
> > > > The most desirable behaviour IMHO is to keep the original type, so:
> > > > - (int8, int8) -> int8
> > > > - (uint16, uint16) -> uint16
> > > >
> > > > Then the question is what happens when the actual overflow occurs.  I
> > > > think this should be directed by a kernel option.  By default an error
> > > > should probably be raised (letting errors pass and silently produce
> > > > erroneous data is wrong), but we might want to allow people to bypass
> > > > overflow checks for speed.
> > > >
> > > > Note that even if overflow detection is enabled, it *should* be possible
> > > > to enable vectorization, e.g. by making overflow detection a separate
> > > > pass (itself vectorizable).
> > > >
> > > > Regards
> > > >
> > > > Antoine.
> > > >
> > > >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Krisztián Szűcs <sz...@gmail.com>.
On Wed, Jun 3, 2020 at 5:52 PM Wes McKinney <we...@gmail.com> wrote:
>
> On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
> <sz...@gmail.com> wrote:
> >
> > From the user perspective I find the following pretty confusing:
> >
> > In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> > Out[1]: array([ 0, -2], dtype=int8)
> >
> > In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> > Out[2]: array([-256,  254], dtype=int16)
> >
> > In my opinion somewhere (on a higher level maybe) we should provide
> > the correct results promoted to a wider type implicitly.
>
> Yes, I agree with you, but I agree that the best place to address this
> is at a higher level rather than having this logic implemented at the
> lowest level (kernels) -- I think database systems handle this during
> logical->physical planning.

It raises another question: where to incorporate the implicit promotions?
// correct me if I'm wrong but these implicit promotions are operation
// dependent and distinct from kernel dispatching issue [1]

The numpy example above can be roughly translated to:
>>> a = pa.array([-128, 127])
>>> pa.compute.add(a, a)
array([ 0, -2]

Which is rather surprising from the user's perspective.

[1] https://issues.apache.org/jira/browse/ARROW-8919
>
> > Clickhouse for example does the type promotion.
> >
> > On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net> wrote:
> > >
> > > On Wed, 3 Jun 2020 10:47:38 -0400
> > > Ben Kietzman <be...@rstudio.com> wrote:
> > > > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > > >
> > > > How should arithmetic kernels handle integer overflow?
> > > >
> > > > The approach currently taken in the linked PR is to promote such that
> > > > overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> > > > uint16)->uint32`.
> > > >
> > > > I'm not sure that's desirable. For one thing this leads to inconsistent
> > > > handling of 64 bit integer types, which are currently allowed to overflow
> > > > since we cannot promote further (NB: that means this kernel includes
> > > > undefined behavior for int64).
> > >
> > > I agree with you.  I would strongly advise against implicit promotion
> > > accross arithmetic operations.  We initially did that in Numba and it
> > > quickly became a can of worms.
> > >
> > > The most desirable behaviour IMHO is to keep the original type, so:
> > > - (int8, int8) -> int8
> > > - (uint16, uint16) -> uint16
> > >
> > > Then the question is what happens when the actual overflow occurs.  I
> > > think this should be directed by a kernel option.  By default an error
> > > should probably be raised (letting errors pass and silently produce
> > > erroneous data is wrong), but we might want to allow people to bypass
> > > overflow checks for speed.
> > >
> > > Note that even if overflow detection is enabled, it *should* be possible
> > > to enable vectorization, e.g. by making overflow detection a separate
> > > pass (itself vectorizable).
> > >
> > > Regards
> > >
> > > Antoine.
> > >
> > >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Wes McKinney <we...@gmail.com>.
On Wed, Jun 3, 2020 at 10:49 AM Krisztián Szűcs
<sz...@gmail.com> wrote:
>
> From the user perspective I find the following pretty confusing:
>
> In [1]: np.array([-128, 127], dtype=np.int8()) * 2
> Out[1]: array([ 0, -2], dtype=int8)
>
> In [2]: np.array([-128, 127], dtype=np.int16()) * 2
> Out[2]: array([-256,  254], dtype=int16)
>
> In my opinion somewhere (on a higher level maybe) we should provide
> the correct results promoted to a wider type implicitly.

Yes, I agree with you, but I agree that the best place to address this
is at a higher level rather than having this logic implemented at the
lowest level (kernels) -- I think database systems handle this during
logical->physical planning.


> Clickhouse for example does the type promotion.
>
> On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net> wrote:
> >
> > On Wed, 3 Jun 2020 10:47:38 -0400
> > Ben Kietzman <be...@rstudio.com> wrote:
> > > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> > >
> > > How should arithmetic kernels handle integer overflow?
> > >
> > > The approach currently taken in the linked PR is to promote such that
> > > overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> > > uint16)->uint32`.
> > >
> > > I'm not sure that's desirable. For one thing this leads to inconsistent
> > > handling of 64 bit integer types, which are currently allowed to overflow
> > > since we cannot promote further (NB: that means this kernel includes
> > > undefined behavior for int64).
> >
> > I agree with you.  I would strongly advise against implicit promotion
> > accross arithmetic operations.  We initially did that in Numba and it
> > quickly became a can of worms.
> >
> > The most desirable behaviour IMHO is to keep the original type, so:
> > - (int8, int8) -> int8
> > - (uint16, uint16) -> uint16
> >
> > Then the question is what happens when the actual overflow occurs.  I
> > think this should be directed by a kernel option.  By default an error
> > should probably be raised (letting errors pass and silently produce
> > erroneous data is wrong), but we might want to allow people to bypass
> > overflow checks for speed.
> >
> > Note that even if overflow detection is enabled, it *should* be possible
> > to enable vectorization, e.g. by making overflow detection a separate
> > pass (itself vectorizable).
> >
> > Regards
> >
> > Antoine.
> >
> >

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Krisztián Szűcs <sz...@gmail.com>.
From the user perspective I find the following pretty confusing:

In [1]: np.array([-128, 127], dtype=np.int8()) * 2
Out[1]: array([ 0, -2], dtype=int8)

In [2]: np.array([-128, 127], dtype=np.int16()) * 2
Out[2]: array([-256,  254], dtype=int16)

In my opinion somewhere (on a higher level maybe) we should provide
the correct results promoted to a wider type implicitly.

Clickhouse for example does the type promotion.

On Wed, Jun 3, 2020 at 5:29 PM Antoine Pitrou <so...@pitrou.net> wrote:
>
> On Wed, 3 Jun 2020 10:47:38 -0400
> Ben Kietzman <be...@rstudio.com> wrote:
> > https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> >
> > How should arithmetic kernels handle integer overflow?
> >
> > The approach currently taken in the linked PR is to promote such that
> > overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> > uint16)->uint32`.
> >
> > I'm not sure that's desirable. For one thing this leads to inconsistent
> > handling of 64 bit integer types, which are currently allowed to overflow
> > since we cannot promote further (NB: that means this kernel includes
> > undefined behavior for int64).
>
> I agree with you.  I would strongly advise against implicit promotion
> accross arithmetic operations.  We initially did that in Numba and it
> quickly became a can of worms.
>
> The most desirable behaviour IMHO is to keep the original type, so:
> - (int8, int8) -> int8
> - (uint16, uint16) -> uint16
>
> Then the question is what happens when the actual overflow occurs.  I
> think this should be directed by a kernel option.  By default an error
> should probably be raised (letting errors pass and silently produce
> erroneous data is wrong), but we might want to allow people to bypass
> overflow checks for speed.
>
> Note that even if overflow detection is enabled, it *should* be possible
> to enable vectorization, e.g. by making overflow detection a separate
> pass (itself vectorizable).
>
> Regards
>
> Antoine.
>
>

Re: [DISCUSS] Add kernel integer overflow handling

Posted by Antoine Pitrou <so...@pitrou.net>.
On Wed, 3 Jun 2020 10:47:38 -0400
Ben Kietzman <be...@rstudio.com> wrote:
> https://github.com/apache/arrow/pull/7341#issuecomment-638241193
> 
> How should arithmetic kernels handle integer overflow?
> 
> The approach currently taken in the linked PR is to promote such that
> overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> uint16)->uint32`.
> 
> I'm not sure that's desirable. For one thing this leads to inconsistent
> handling of 64 bit integer types, which are currently allowed to overflow
> since we cannot promote further (NB: that means this kernel includes
> undefined behavior for int64).

I agree with you.  I would strongly advise against implicit promotion
accross arithmetic operations.  We initially did that in Numba and it
quickly became a can of worms.

The most desirable behaviour IMHO is to keep the original type, so:
- (int8, int8) -> int8
- (uint16, uint16) -> uint16

Then the question is what happens when the actual overflow occurs.  I
think this should be directed by a kernel option.  By default an error
should probably be raised (letting errors pass and silently produce
erroneous data is wrong), but we might want to allow people to bypass
overflow checks for speed.

Note that even if overflow detection is enabled, it *should* be possible
to enable vectorization, e.g. by making overflow detection a separate
pass (itself vectorizable).

Regards

Antoine.



Re: [DISCUSS] Add kernel integer overflow handling

Posted by Wes McKinney <we...@gmail.com>.
What do open source analytic database systems do? I don't think we
should deviate from the behavior of these systems. For example, you
can see that Apache Impala uses unsigned arithmetic on signed integers

https://github.com/apache/impala/blob/5c69e7ba583297dc886652ac5952816882b928af/be/src/exprs/operators-ir.cc#L38

On Wed, Jun 3, 2020 at 9:47 AM Ben Kietzman <be...@rstudio.com> wrote:
>
> https://github.com/apache/arrow/pull/7341#issuecomment-638241193
>
> How should arithmetic kernels handle integer overflow?
>
> The approach currently taken in the linked PR is to promote such that
> overflow will not occur, for example `(int8, int8)->int16` and `(uint16,
> uint16)->uint32`.
>
> I'm not sure that's desirable. For one thing this leads to inconsistent
> handling of 64 bit integer types, which are currently allowed to overflow
> since we cannot promote further (NB: that means this kernel includes
> undefined behavior for int64).
>
> There are a few other approaches we could take (ordered by personal
> preference):
>
>    - define explicit overflow behavior for signed integer operands (for
>    example if we declared that add(i8(a), i8(b)) will always be equivalent
>    to i8(i16(a) + i16(b)) then we could instantiate only unsigned addition
>    kernels)
>    - raise an error on signed overflow
>    - provide ArithmeticOptions::overflow_behavior and allow users to choose
>    between these
>    - require users to pass arguments which will not overflow