Re: [VOTE] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-07-08 Thread Alexey Leonov-Vendrovskiy
Thank you all!
The voting is closed now.
I've posted results [1].

[1] https://lists.apache.org/thread/mx3qd5qgyjtqk66p6sv639g777lm0jc6

Thanks,
Alexey

On Wed, Jul 3, 2024 at 6:39 PM Yuepeng Pan  wrote:

> +1 (non-binding)
>
> Best regards,
>
> Yuepeng Pan
>
>
>
>
>
> At 2024-07-03 01:46:13, "Sergey Nuyanzin"  wrote:
> >Thanks for driving this
> >
> >+1 (binding)
> >
> >On Tue, Jul 2, 2024, 11:21 Martijn Visser 
> wrote:
> >
> >> +1 (binding)
> >>
> >> On Mon, Jul 1, 2024 at 7:00 PM Jim Hughes  >
> >> wrote:
> >>
> >> > Hi Alexey,
> >> >
> >> > +1 (non-binding)
> >> >
> >> > I'm looking forward to parity between streaming and batch bound for
> >> > compiled plans!
> >> >
> >> > Cheers,
> >> >
> >> > Jim
> >> >
> >> > On Mon, Jul 1, 2024 at 12:55 PM Alexey Leonov-Vendrovskiy <
> >> > vendrov...@gmail.com> wrote:
> >> >
> >> > > Hello everyone,
> >> > >
> >> > > We had a good discussion of FLIP-456: CompiledPlan support for Batch
> >> > > Execution Mode [1]. Discussion thread is here: [2].
> >> > >
> >> > > Let's start voting on it. The vote will be open for at least 72
> >> > > hours unless there is an objection or insufficient votes. The FLIP
> will
> >> > be
> >> > > considered accepted if 3 binding votes (from active committers
> >> according
> >> > to
> >> > > the Flink bylaws [3]) are gathered by the community.
> >> > >
> >> > > Thanks,
> >> > > Alexey
> >> > >
> >> > > [1]
> >> > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-456%3A+CompiledPlan+support+for+Batch+Execution+Mode
> >> > > [2]
> https://lists.apache.org/thread/7gpyqvdnnbjwbh3vbk6b0pj38l91crvv
> >> > > [3]
> >> > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/Flink+Bylaws#FlinkBylaws-Approvals
> >> > > <
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/Flink+Bylaws#FlinkBylaws-Approvals](https://cwiki.apache.org/confluence/display/FLINK/Flink+Bylaws%23FlinkBylaws-Approvals)
> >> > > >
> >> > >
> >> >
> >>
>


[RESULT][VOTE] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-07-08 Thread Alexey Leonov-Vendrovskiy
Hello everyone,

The vote [1] for FLIP-456 [2] is over. The number of required binding votes
(3) was reached (total: 5, binding: 3, non-binding: 2). No objections were
raised.

Binding:

   - Martijn Visser
   - Sergey Nuyanzin
   - Timo Walther

Non-binding:

   - Jim Hughes
   - Yuepeng Pan


PRs will be prepared. Thank you everyone who participated!

All the best,
Alexey

[1] https://lists.apache.org/thread/2ycws2zplcyd1k25pn1ljmxvg5lgd8r2
[2]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-456%3A+CompiledPlan+support+for+Batch+Execution+Mode


[VOTE] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-07-01 Thread Alexey Leonov-Vendrovskiy
Hello everyone,

We had a good discussion of FLIP-456: CompiledPlan support for Batch
Execution Mode [1]. Discussion thread is here: [2].

Let's start voting on it. The vote will be open for at least 72
hours unless there is an objection or insufficient votes. The FLIP will be
considered accepted if 3 binding votes (from active committers according to
the Flink bylaws [3]) are gathered by the community.

Thanks,
Alexey

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-456%3A+CompiledPlan+support+for+Batch+Execution+Mode
[2] https://lists.apache.org/thread/7gpyqvdnnbjwbh3vbk6b0pj38l91crvv
[3]
https://cwiki.apache.org/confluence/display/FLINK/Flink+Bylaws#FlinkBylaws-Approvals



Re: [DISCUSSION] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-06-24 Thread Alexey Leonov-Vendrovskiy
Hey all, starting a vote in 24 hours. Unless there are more points to
discuss.

Thanks,
Alexey

On Mon, Jun 17, 2024 at 8:28 AM Alexey Leonov-Vendrovskiy <
vendrov...@gmail.com> wrote:

> Done. There was some accidental removal during editing.
>
> Thanks,
> Alexey
>
> On Mon, Jun 17, 2024 at 5:23 AM Jim Hughes 
> wrote:
>
>> Hi Alexey,
>>
>> Looks reasonable.  As a note, I think this sentence is missing some words:
>>
>> "Due to batch queries nature, we do not expect the jobs to run forever
>> (days, months, years) – unlike from . "
>>
>> Can you update it?
>>
>> Cheers,
>>
>> Jim
>>
>> On Mon, Jun 17, 2024 at 3:16 AM Alexey Leonov-Vendrovskiy <
>> vendrov...@gmail.com> wrote:
>>
>> > Thanks Timo and Jim! Added a few sentences to the FLIP to cover your
>> > points.
>> > -Alexey
>> >
>> > On Mon, Jun 10, 2024 at 11:23 PM Timo Walther 
>> wrote:
>> >
>> > > Hi Alexey,
>> > >
>> > > thanks for proposing this FLIP. It is a nice continuation of the
>> vision
>> > > we had for CompiledPlan when writing and implementing FLIP-190. The
>> > > whole stack is prepared for serializing BatchExecNodes as well so it
>> > > shouldn't be too hard to make this a reality.
>> > >
>> > >  > I think the FLIP should be clear on the backwards support strategy
>> > >  > here. The strategy for streaming is "forever".  This may be the
>> most
>> > >  > interesting part of the FLIP to discuss.
>> > >
>> > > I agree with Jim. We shouldn't put too much burden on us (the Flink
>> > > community). BatchExecNodes can evolve quicker than StreamExecNodes as
>> > > the state component isn't an issue. Backwards compatibility of 2-3
>> Flink
>> > > versions and at least 1 year of time should be enough for batch
>> > > infrastructure to update. Of course we should avoid breaking changes
>> > > whenever possible. This should be written down in the FLIP.
>> > >
>> > > Regards,
>> > > Timo
>> > >
>> > >
>> > >
>> > >
>> > > On 07.06.24 23:10, Jim Hughes wrote:
>> > > > Hi Alexey,
>> > > >
>> > > > Responses inline below:
>> > > >
>> > > > On Mon, May 13, 2024 at 7:18 PM Alexey Leonov-Vendrovskiy <
>> > > > vendrov...@gmail.com> wrote:
>> > > >
>> > > >> Thanks Jim.
>> > > >>
>> > > >>> 1. For the testing, I'd call the tests "execution" tests rather
>> than
>> > > >>> "restore" tests.  For streaming execution, restore tests have the
>> > > >> compiled
>> > > >>> plan and intermediate state; the tests verify that those can work
>> > > >> together
>> > > >>> and continue processing.
>> > > >>
>> > > >> Agree that we don't need to store and restore the intermediate
>> state.
>> > So
>> > > >> the most critical part is that the CompiledPlan for batch can be
>> > > executed.
>> > > >>
>> > > >
>> > > > On the FLIP, can you be more specific about what we are checking
>> during
>> > > > execution?  I'd suggest that `executeSql(_)` and
>> > > > `executePlan(compilePlanSql(_))` should be compared.
>> > > >
>> > > >
>> > > >> 2. The FLIP implicitly suggests "completeness tests" (to use
>> > FLIP-190's
>> > > >>> words).  Do we need "change detection tests"?  I'm a little
>> unsure if
>> > > >> that
>> > > >>> is presently happening in an automatic way for streaming
>> operators.
>> > > >>
>> > > >>
>> > > >>   We might need to elaborate more on this, but the idea is that  we
>> > > need to
>> > > >> make sure that compiled plans created by an older version of SQL
>> > Planner
>> > > >> are executable on newer runtimes.
>> > > >>
>> > > >> 3.  Can we remove old versions of batch operators eventually?  Or
>> do
>> > we
>> > > >>> need to keep them forever like we would for streaming operators?
>> > > >>>
>> > > >>
>

Re: [DISCUSSION] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-06-17 Thread Alexey Leonov-Vendrovskiy
Done. There was some accidental removal during editing.

Thanks,
Alexey

On Mon, Jun 17, 2024 at 5:23 AM Jim Hughes 
wrote:

> Hi Alexey,
>
> Looks reasonable.  As a note, I think this sentence is missing some words:
>
> "Due to batch queries nature, we do not expect the jobs to run forever
> (days, months, years) – unlike from . "
>
> Can you update it?
>
> Cheers,
>
> Jim
>
> On Mon, Jun 17, 2024 at 3:16 AM Alexey Leonov-Vendrovskiy <
> vendrov...@gmail.com> wrote:
>
> > Thanks Timo and Jim! Added a few sentences to the FLIP to cover your
> > points.
> > -Alexey
> >
> > On Mon, Jun 10, 2024 at 11:23 PM Timo Walther 
> wrote:
> >
> > > Hi Alexey,
> > >
> > > thanks for proposing this FLIP. It is a nice continuation of the vision
> > > we had for CompiledPlan when writing and implementing FLIP-190. The
> > > whole stack is prepared for serializing BatchExecNodes as well so it
> > > shouldn't be too hard to make this a reality.
> > >
> > >  > I think the FLIP should be clear on the backwards support strategy
> > >  > here. The strategy for streaming is "forever".  This may be the most
> > >  > interesting part of the FLIP to discuss.
> > >
> > > I agree with Jim. We shouldn't put too much burden on us (the Flink
> > > community). BatchExecNodes can evolve quicker than StreamExecNodes as
> > > the state component isn't an issue. Backwards compatibility of 2-3
> Flink
> > > versions and at least 1 year of time should be enough for batch
> > > infrastructure to update. Of course we should avoid breaking changes
> > > whenever possible. This should be written down in the FLIP.
> > >
> > > Regards,
> > > Timo
> > >
> > >
> > >
> > >
> > > On 07.06.24 23:10, Jim Hughes wrote:
> > > > Hi Alexey,
> > > >
> > > > Responses inline below:
> > > >
> > > > On Mon, May 13, 2024 at 7:18 PM Alexey Leonov-Vendrovskiy <
> > > > vendrov...@gmail.com> wrote:
> > > >
> > > >> Thanks Jim.
> > > >>
> > > >>> 1. For the testing, I'd call the tests "execution" tests rather
> than
> > > >>> "restore" tests.  For streaming execution, restore tests have the
> > > >> compiled
> > > >>> plan and intermediate state; the tests verify that those can work
> > > >> together
> > > >>> and continue processing.
> > > >>
> > > >> Agree that we don't need to store and restore the intermediate
> state.
> > So
> > > >> the most critical part is that the CompiledPlan for batch can be
> > > executed.
> > > >>
> > > >
> > > > On the FLIP, can you be more specific about what we are checking
> during
> > > > execution?  I'd suggest that `executeSql(_)` and
> > > > `executePlan(compilePlanSql(_))` should be compared.
> > > >
> > > >
> > > >> 2. The FLIP implicitly suggests "completeness tests" (to use
> > FLIP-190's
> > > >>> words).  Do we need "change detection tests"?  I'm a little unsure
> if
> > > >> that
> > > >>> is presently happening in an automatic way for streaming operators.
> > > >>
> > > >>
> > > >>   We might need to elaborate more on this, but the idea is that  we
> > > need to
> > > >> make sure that compiled plans created by an older version of SQL
> > Planner
> > > >> are executable on newer runtimes.
> > > >>
> > > >> 3.  Can we remove old versions of batch operators eventually?  Or do
> > we
> > > >>> need to keep them forever like we would for streaming operators?
> > > >>>
> > > >>
> > > >> We could have deprecation paths for old operator nodes in some
> cases.
> > > It is
> > > >> a matter of the time window: what could be practical the "time
> > distance"
> > > >> between query planner and flink runtime against which the query
> query
> > > can
> > > >> be resubmitted.
> > > >> Note, here we don't have continuous queries, so there is always an
> > > option
> > > >> to "re-plan" the original SQL query text into a newer version of the
> > > >> CompiledPlan.
> > &

Re: [DISCUSSION] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-06-17 Thread Alexey Leonov-Vendrovskiy
Thanks Timo and Jim! Added a few sentences to the FLIP to cover your points.
-Alexey

On Mon, Jun 10, 2024 at 11:23 PM Timo Walther  wrote:

> Hi Alexey,
>
> thanks for proposing this FLIP. It is a nice continuation of the vision
> we had for CompiledPlan when writing and implementing FLIP-190. The
> whole stack is prepared for serializing BatchExecNodes as well so it
> shouldn't be too hard to make this a reality.
>
>  > I think the FLIP should be clear on the backwards support strategy
>  > here. The strategy for streaming is "forever".  This may be the most
>  > interesting part of the FLIP to discuss.
>
> I agree with Jim. We shouldn't put too much burden on us (the Flink
> community). BatchExecNodes can evolve quicker than StreamExecNodes as
> the state component isn't an issue. Backwards compatibility of 2-3 Flink
> versions and at least 1 year of time should be enough for batch
> infrastructure to update. Of course we should avoid breaking changes
> whenever possible. This should be written down in the FLIP.
>
> Regards,
> Timo
>
>
>
>
> On 07.06.24 23:10, Jim Hughes wrote:
> > Hi Alexey,
> >
> > Responses inline below:
> >
> > On Mon, May 13, 2024 at 7:18 PM Alexey Leonov-Vendrovskiy <
> > vendrov...@gmail.com> wrote:
> >
> >> Thanks Jim.
> >>
> >>> 1. For the testing, I'd call the tests "execution" tests rather than
> >>> "restore" tests.  For streaming execution, restore tests have the
> >> compiled
> >>> plan and intermediate state; the tests verify that those can work
> >> together
> >>> and continue processing.
> >>
> >> Agree that we don't need to store and restore the intermediate state. So
> >> the most critical part is that the CompiledPlan for batch can be
> executed.
> >>
> >
> > On the FLIP, can you be more specific about what we are checking during
> > execution?  I'd suggest that `executeSql(_)` and
> > `executePlan(compilePlanSql(_))` should be compared.
> >
> >
> >> 2. The FLIP implicitly suggests "completeness tests" (to use FLIP-190's
> >>> words).  Do we need "change detection tests"?  I'm a little unsure if
> >> that
> >>> is presently happening in an automatic way for streaming operators.
> >>
> >>
> >>   We might need to elaborate more on this, but the idea is that  we
> need to
> >> make sure that compiled plans created by an older version of SQL Planner
> >> are executable on newer runtimes.
> >>
> >> 3.  Can we remove old versions of batch operators eventually?  Or do we
> >>> need to keep them forever like we would for streaming operators?
> >>>
> >>
> >> We could have deprecation paths for old operator nodes in some cases.
> It is
> >> a matter of the time window: what could be practical the "time distance"
> >> between query planner and flink runtime against which the query query
> can
> >> be resubmitted.
> >> Note, here we don't have continuous queries, so there is always an
> option
> >> to "re-plan" the original SQL query text into a newer version of the
> >> CompiledPlan.
> >> With this in mind, a time window of 1yr+ would allow deprecation of
> older
> >> batch exec nodes, though I don't see this as a frequent event.
> >>
> >
> > As I read the JavaDocs for `TableEnvironment.loadPlan`, it looks like the
> > compiled plan ought to be sufficient to run a job at a later time.
> >
> > I think the FLIP should be clear on the backwards support strategy here.
> > The strategy for streaming is "forever".  This may be the most
> interesting
> > part of the FLIP to discuss.
> >
> > Can you let us know when you've updated the FLIP?
> >
> > Cheers,
> >
> > Jim
> >
> >
> >> -Alexey
> >>
> >>
> >>
> >> On Mon, May 13, 2024 at 1:52 PM Jim Hughes  >
> >> wrote:
> >>
> >>> Hi Alexey,
> >>>
> >>> After some thought, I have a question about deprecations:
> >>>
> >>> 3.  Can we remove old versions of batch operators eventually?  Or do we
> >>> need to keep them forever like we would for streaming operators?
> >>>
> >>> Cheers,
> >>>
> >>> Jim
> >>>
> >>> On Thu, May 9, 2024 at 11:29 AM Jim Hughes 
> wrote:
> >>>
> >>>> Hi Alexey,
> >>>>
>

Re: [DISCUSSION] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-05-13 Thread Alexey Leonov-Vendrovskiy
Thanks Jim.



> 1. For the testing, I'd call the tests "execution" tests rather than
> "restore" tests.  For streaming execution, restore tests have the compiled
> plan and intermediate state; the tests verify that those can work together
> and continue processing.


Agree that we don't need to store and restore the intermediate state. So
the most critical part is that the CompiledPlan for batch can be executed.

2. The FLIP implicitly suggests "completeness tests" (to use FLIP-190's
> words).  Do we need "change detection tests"?  I'm a little unsure if that
> is presently happening in an automatic way for streaming operators.


 We might need to elaborate more on this, but the idea is that  we need to
make sure that compiled plans created by an older version of SQL Planner
are executable on newer runtimes.

3.  Can we remove old versions of batch operators eventually?  Or do we
> need to keep them forever like we would for streaming operators?
>

We could have deprecation paths for old operator nodes in some cases. It is
a matter of the time window: what could be practical the "time distance"
between query planner and flink runtime against which the query query can
be resubmitted.
Note, here we don't have continuous queries, so there is always an option
to "re-plan" the original SQL query text into a newer version of the
CompiledPlan.
With this in mind, a time window of 1yr+ would allow deprecation of older
batch exec nodes, though I don't see this as a frequent event.

-Alexey



On Mon, May 13, 2024 at 1:52 PM Jim Hughes 
wrote:

> Hi Alexey,
>
> After some thought, I have a question about deprecations:
>
> 3.  Can we remove old versions of batch operators eventually?  Or do we
> need to keep them forever like we would for streaming operators?
>
> Cheers,
>
> Jim
>
> On Thu, May 9, 2024 at 11:29 AM Jim Hughes  wrote:
>
> > Hi Alexey,
> >
> > Overall, the FLIP looks good and makes sense to me.
> >
> > 1. For the testing, I'd call the tests "execution" tests rather than
> > "restore" tests.  For streaming execution, restore tests have the
> compiled
> > plan and intermediate state; the tests verify that those can work
> together
> > and continue processing.
> >
> > For batch execution, I think we just want that all existing compiled
> plans
> > can be executed in future versions.
> >
> > 2. The FLIP implicitly suggests "completeness tests" (to use FLIP-190's
> > words).  Do we need "change detection tests"?  I'm a little unsure if
> that
> > is presently happening in an automatic way for streaming operators.
> >
> > In RestoreTestBase, generateTestSetupFiles is disabled and has to be run
> > manually when tests are being written.
> >
> > Cheers,
> >
> > Jim
> >
> > On Tue, May 7, 2024 at 5:11 AM Paul Lam  wrote:
> >
> >> Hi Alexey,
> >>
> >> Thanks a lot for bringing up the discussion. I’m big +1 for the FLIP.
> >>
> >> I suppose the goal doesn’t involve the interchangeability of json plans
> >> between batch mode and streaming mode, right?
> >> In other words, a json plan compiled in a batch program can’t be run in
> >> streaming mode without a migration (which is not yet supported).
> >>
> >> Best,
> >> Paul Lam
> >>
> >> > 2024年5月7日 14:38,Alexey Leonov-Vendrovskiy  写道:
> >> >
> >> > Hi everyone,
> >> >
> >> > PTAL at the proposed FLIP-456: CompiledPlan support for Batch
> Execution
> >> > Mode. It is pretty self-describing.
> >> >
> >> > Any thoughts are welcome!
> >> >
> >> > Thanks,
> >> > Alexey
> >> >
> >> > [1]
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-456%3A+CompiledPlan+support+for+Batch+Execution+Mode
> >> > .
> >>
> >>
>


Re: [DISCUSSION] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-05-07 Thread Alexey Leonov-Vendrovskiy
Hey Paul,

Yes, no interchangeability. Just a wire-through for more uniformity.

Thanks,
Alexey

On Tue, May 7, 2024 at 2:10 AM Paul Lam  wrote:

> Hi Alexey,
>
> Thanks a lot for bringing up the discussion. I’m big +1 for the FLIP.
>
> I suppose the goal doesn’t involve the interchangeability of json plans
> between batch mode and streaming mode, right?
> In other words, a json plan compiled in a batch program can’t be run in
> streaming mode without a migration (which is not yet supported).
>
> Best,
> Paul Lam
>
> > 2024年5月7日 14:38,Alexey Leonov-Vendrovskiy  写道:
> >
> > Hi everyone,
> >
> > PTAL at the proposed FLIP-456: CompiledPlan support for Batch Execution
> > Mode. It is pretty self-describing.
> >
> > Any thoughts are welcome!
> >
> > Thanks,
> > Alexey
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-456%3A+CompiledPlan+support+for+Batch+Execution+Mode
> > .
>
>


[DISCUSSION] FLIP-456: CompiledPlan support for Batch Execution Mode

2024-05-07 Thread Alexey Leonov-Vendrovskiy
Hi everyone,

PTAL at the proposed FLIP-456: CompiledPlan support for Batch Execution
Mode. It is pretty self-describing.

Any thoughts are welcome!

Thanks,
Alexey

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-456%3A+CompiledPlan+support+for+Batch+Execution+Mode
.


[jira] [Created] (FLINK-35241) Support SQL FLOOR and CEIL functions with SECOND and MINUTE for TIMESTAMP_TLZ

2024-04-25 Thread Alexey Leonov-Vendrovskiy (Jira)
Alexey Leonov-Vendrovskiy created FLINK-35241:
-

 Summary: Support SQL FLOOR and CEIL functions with SECOND and 
MINUTE for TIMESTAMP_TLZ
 Key: FLINK-35241
 URL: https://issues.apache.org/jira/browse/FLINK-35241
 Project: Flink
  Issue Type: Bug
  Components: Table SQL / API
Reporter: Alexey Leonov-Vendrovskiy


We need a fix for both SECOND and MINUTE.

The following query doesn't work:


{code:java}
SELECT
  FLOOR(
      CAST(TIMESTAMP '2024-04-25 17:19:42.654' AS TIMESTAMP_LTZ(3))
  TO MINUTE) {code}


These two queries work:


{code:java}
SELECT
  FLOOR(
      CAST(TIMESTAMP '2024-04-25 17:19:42.654' AS TIMESTAMP_LTZ(3))
  TO HOUR) {code}
 
{code:java}
SELECT
  FLOOR(
      TIMESTAMP '2024-04-25 17:19:42.654'
  TO MINUTE) {code}


Stack trace:

For the first not working query:
{code:java}
Caused by: io.confluent.flink.table.utils.CleanedException: 
org.codehaus.commons.compiler.CompileException: Line 41, Column 69: No 
applicable constructor/method found for actual parameters 
"org.apache.flink.table.data.TimestampData, 
org.apache.flink.table.data.TimestampData"; candidates are: "public static long 
org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(long, long)", 
"public static float 
org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(float)", 
"public static org.apache.flink.table.data.DecimalData 
org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(org.apache.flink.table.data.DecimalData)",
 "public static int 
org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(int, int)", 
"public static double 
org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(double)"
at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:13080)
at 
org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9646)
at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9506)
at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9422)
at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5263)
... {code}
 
 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: Making CollectSinkFunction to wait till all results are consumed

2024-02-19 Thread Alexey Leonov-Vendrovskiy
Hey, Piotrek,

Thanks for the response and thoughts!

I don't think it is possible to tell the difference between different cases
-- when someone would not want the results back from the instantiated
CollectSinkFunction.
It is not clear why this would be the case, unless it is some test flow
that explicitly does unusual things.

Regards,
-Alexey


On Wed, Feb 14, 2024 at 12:27 AM Piotr Nowojski 
wrote:

> Hi!
>
> Interesting catch. I think the current master branch behaviour is broken.
> The chance to lose some records on `endInput`
> is simply a critical bug. The limited buffer size is still problematic, as
> it can surprise users. Having said that, the newly
> proposed behaviour without any buffer is also problematic.
>
> Maybe whenever the user wants to collect the results, before or when
> calling `env.execute()` we should spawn a new
> thread that would asynchronously collect results from the
> `CollectSinkFunction`? I'm not sure but maybe hooking this
> logic up to whenever `CollectStreamSink` is being used is the way to go?
>
> One thing I'm not sure about is whether there are scenarios/existing code
> paths where someone wants to use the
> `CollectSinkFunction` but doesn't want the results to be read
> automatically? And if there are, can we tell them apart?
>
> Best,
> Piotrek
>
> pon., 12 lut 2024 o 08:48 Alexey Leonov-Vendrovskiy 
> napisał(a):
>
> > Hey all,
> >
> > We propose to slightly change the behavior of the CollectSinkFunction
> > <
> >
> https://github.com/apache/flink/blob/6f4d31f1b79afbde6c093b5d40ac83fe7524e303/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java#L129
> > >
> > and make it wait till all the result from the buffer is consumed by the
> > client, before shutting it down.
> >
> > Overall protocol and all the other behavior stay the same.
> >
> > This would be a way to guarantee result availability upon the job
> > completion. Today, the tail of the result is stored in an accumulator,
> and
> > gets stored in the job manager. There is an opportunity for this part of
> > the result to get lost, after the job is claimed to be
> > successfully "completed". Waiting till all the results are consumed while
> > the job is running is a natural way to achieve availability. Once the job
> > is done, we are certain all the results are consumed.
> >
> > This change would be achieved by overriding the endInput() method
> > in CollectSinkOperator, and passing the call to CollectSinkFunction to
> wait
> > till the buffer is empty.
> >
> > The old behavior could be enabled via a configuration flag (to be added).
> >
> > A notable side-effect of the change is that any invocation
> > of StreamExecutionEnvironment.execute() (synchronous execution) with a
> > pipeline with CollectSinkFunction in it, would effectively block waiting
> > for the results to get consumed. This would require running the consumer
> on
> > a different thread. Though note, it is* already the case* when the result
> > is larger that what can fit into the CollectSinkFunction's buffer.  Take
> a
> > look at flink-end-to-end-tests/test-scripts/test_quickstarts.sh in the
> > current state of the repo: if we change the parameter numRecords to be
> > 1,000,000, the test locks and waits forever. So, the only difference with
> > the change would be that in similar setups it would wait on any buffer
> size
> > > 0. It makes behavior consistent for results of any non-zero size.
> >
> >
> > Let me know your thoughts.
> >
> > Thanks,
> > Alexey
> >
>


Making CollectSinkFunction to wait till all results are consumed

2024-02-11 Thread Alexey Leonov-Vendrovskiy
Hey all,

We propose to slightly change the behavior of the CollectSinkFunction

and make it wait till all the result from the buffer is consumed by the
client, before shutting it down.

Overall protocol and all the other behavior stay the same.

This would be a way to guarantee result availability upon the job
completion. Today, the tail of the result is stored in an accumulator, and
gets stored in the job manager. There is an opportunity for this part of
the result to get lost, after the job is claimed to be
successfully "completed". Waiting till all the results are consumed while
the job is running is a natural way to achieve availability. Once the job
is done, we are certain all the results are consumed.

This change would be achieved by overriding the endInput() method
in CollectSinkOperator, and passing the call to CollectSinkFunction to wait
till the buffer is empty.

The old behavior could be enabled via a configuration flag (to be added).

A notable side-effect of the change is that any invocation
of StreamExecutionEnvironment.execute() (synchronous execution) with a
pipeline with CollectSinkFunction in it, would effectively block waiting
for the results to get consumed. This would require running the consumer on
a different thread. Though note, it is* already the case* when the result
is larger that what can fit into the CollectSinkFunction's buffer.  Take a
look at flink-end-to-end-tests/test-scripts/test_quickstarts.sh in the
current state of the repo: if we change the parameter numRecords to be
1,000,000, the test locks and waits forever. So, the only difference with
the change would be that in similar setups it would wait on any buffer size
> 0. It makes behavior consistent for results of any non-zero size.


Let me know your thoughts.

Thanks,
Alexey


Re: [VOTE] FLIP-387: Support named parameters for functions and call procedures

2024-01-05 Thread Alexey Leonov-Vendrovskiy
Thanks for starting the vote!
Do you mind adding a link from the FLIP to this thread?

Thanks,
Alexey

On Thu, Jan 4, 2024 at 6:48 PM Feng Jin  wrote:

> Hi everyone
>
> Thanks for all the feedback about the FLIP-387: Support named parameters
> for functions and call procedures [1] [2] .
>
> I'd like to start a vote for it. The vote will be open for at least 72
> hours(excluding weekends,until Jan 10, 12:00AM GMT) unless there is an
> objection or an insufficient number of votes.
>
>
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-387%3A+Support+named+parameters+for+functions+and+call+procedures
> [2] https://lists.apache.org/thread/bto7mpjvcx7d7k86owb00dwrm65jx8cn
>
>
> Best,
> Feng Jin
>


Re: [DISCUSS] FLIP-348: Support System Columns in SQL and Table API

2023-08-28 Thread Alexey Leonov-Vendrovskiy
tor and Table API.
> > > >>>
> > > >>> Queries such as `SELECT * FROM (SELECT $rowtime, * FROM t);` would
> > show
> > > >>> $rowtime as the expand star has only a special case when in the
> scope
> > > >>> for `FROM t`. All further subqueries treat it as a regular column.
> > > >>>
> > > >>> 4) Built-in defined pseudo-column "$rowtime"
> > > >>>
> > > >>>   > Did you consider making it as a built-in defined pseudo-column
> > > >>> "$rowtime" which returns the time attribute value (if exists) or
> null
> > > >>> (if non-exists) for every table/query, and pseudo-column
> "$proctime"
> > > >>> always returns PROCTIME() value for each table/query
> > > >>>
> > > >>> Built-in pseudo-columns mean that connector or catalog providers
> need
> > > >>> consensus in Flink which pseudo-columns should be built-in. We
> should
> > > >>> keep the concept generic and let platform providers decide which
> > pseudo
> > > >>> columns to expose. $rowtime might be obvious but others such as
> > > >>> $partition or $offset are tricky to get consensus as every external
> > > >>> connector works differently. Also a connector might want to expose
> > > >>> different time semantics (such as ingestion time).
> > > >>>
> > > >>> 5) Any operator can introduce system (psedo) columns.
> > > >>>
> > > >>> This is clearly out of scope for this FLIP. The implementation
> effort
> > > >>> would be huge and could introduce a lot of bugs.
> > > >>>
> > > >>> 6) "Metadata Key Prefix Constraint" which is still a little complex
> > > >>>
> > > >>> Another option could be to drop the naming pattern constraint. We
> > could
> > > >>> make it configurable that METADATA VIRTUAL columns are never
> selected
> > > by
> > > >>> default in SELECT * or visible in DESCRIBE. This would further
> > simplify
> > > >>> the FLIP and esp lower the impact on the planner and all
> interfaces.
> > > >>>
> > > >>> What do you think about this? We could introduce a flag:
> > > >>>
> > > >>> table.expand-metadata-columns (better name to be defined)
> > > >>>
> > > >>> This way we don't need to introduce the concept of system columns
> > yet,
> > > >>> but can still offer similar functionality with minimal overhead in
> > the
> > > >>> code base.
> > > >>>
> > > >>> Regards,
> > > >>> Timo
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > > >>> On 04.08.23 23:06, Alexey Leonov-Vendrovskiy wrote:
> > > >>>> Looks like both kinds of system columns can converge.
> > > >>>> We can say that any operator can introduce system (psedo) columns.
> > > >>>>
> > > >>>> cc Eugene who is also interested in the subject.
> > > >>>>
> > > >>>> On Wed, Aug 2, 2023 at 1:03 AM Paul Lam 
> > > wrote:
> > > >>>>
> > > >>>>> Hi Timo,
> > > >>>>>
> > > >>>>> Thanks for starting the discussion! System columns are no doubt a
> > > >>>>> good boost on Flink SQL’s usability, and I see the feedbacks are
> > > >>>>> mainly concerns about the accessibility of system columns.
> > > >>>>>
> > > >>>>> I think most of the concerns could be solved by clarifying the
> > > >>>>> ownership of the system columns. Different from databases like
> > > >>>>> Oracle/BigQuery/PG who owns the data/metadata, Flink uses the
> > > >>>>> data/metadata from external systems. That means Flink could
> > > >>>>> have 2 kinds of system columns (take ROWID for example):
> > > >>>>>
> > > >>>>> 1. system columns provided by external systems via catalogs, such
> > > >>>>>   as ROWID from the original system.
> > > >>>>> 2. system columns generated by Flink, such as ROWID generated by
&g

Re: [DISCUSS] FLIP-348: Support System Columns in SQL and Table API

2023-08-04 Thread Alexey Leonov-Vendrovskiy
Looks like both kinds of system columns can converge.
We can say that any operator can introduce system (psedo) columns.

cc Eugene who is also interested in the subject.

On Wed, Aug 2, 2023 at 1:03 AM Paul Lam  wrote:

> Hi Timo,
>
> Thanks for starting the discussion! System columns are no doubt a
> good boost on Flink SQL’s usability, and I see the feedbacks are
> mainly concerns about the accessibility of system columns.
>
> I think most of the concerns could be solved by clarifying the
> ownership of the system columns. Different from databases like
> Oracle/BigQuery/PG who owns the data/metadata, Flink uses the
> data/metadata from external systems. That means Flink could
> have 2 kinds of system columns (take ROWID for example):
>
> 1. system columns provided by external systems via catalogs, such
> as ROWID from the original system.
> 2. system columns generated by Flink, such as ROWID generated by
> Flink itself.
>
> IIUC, the FLIP is proposing the 1st approach: the catalog defines what
> system columns to provide, and Flink treats them as normal columns
> with a special naming pattern.
>
> On the other hand, Jark is proposing the 2nd one: the system columns
> are defined and owned by Flink, and can be inferred from external
> systems. Therefore, system columns should be predefined by Flink,
> and optionally implemented by the catalogs.
>
> Personally, I’m in favor of the 2nd approach, because it makes the
> system columns very accessible and more aligned across the catalogs.
>
> BTW, I second Alexey that systems columns should not be shown with
> DESCRIBE statements.
>
> WDYT? Thanks!
>
> Best,
> Paul Lam
>
> > 2023年7月31日 23:54,Jark Wu  写道:
> >
> > Hi Timo,
> >
> > Thanks for your proposal. I think this is a nice feature for users and I
> > prefer option 3.
> >
> > I only have one concern about the concept of pseudo-column or
> > system-column,
> > because this is the first time we introduce it in Flink SQL. The
> > confusion is similar to the
> > question of Benchao and Sergey about the propagation of pseudo-column.
> >
> > From my understanding, a pseudo-column can be get from an arbitrary
> query,
> > just similar to
> > ROWNUM in Oracle[1], such as :
> >
> > SELECT *
> > FROM (SELECT * FROM employees ORDER BY employee_id)
> > WHERE ROWNUM < 11;
> >
> > However, IIUC, the proposed "$rowtime" pseudo-column can only be got from
> > the physical table
> > and can't be got from queries even if the query propagates the rowtime
> > attribute. There was also
> > a discussion about adding a pseudo-column "_proctime" [2] to make lookup
> > join easier to use
> > which can be got from arbitrary queries. That "_proctime" may conflict
> with
> > the proposed
> > pseudo-column concept.
> >
> > Did you consider making it as a built-in defined pseudo-column "$rowtime"
> > which returns the
> > time attribute value (if exists) or null (if non-exists) for every
> > table/query, and pseudo-column
> > "$proctime" always returns PROCTIME() value for each table/query. In this
> > way, catalogs only need
> > to provide a default rowtime attribute and users can get it in the same
> > way. And we don't need
> > to introduce the contract interface of "Metadata Key Prefix Constraint"
> > which is still a little complex
> > for users and devs to understand.
> >
> > Best,
> > Jark
> >
> > [1]:
> >
> https://docs.oracle.com/cd/E11882_01/server.112/e41084/pseudocolumns009.htm#SQLRF00255
> > [2]: https://lists.apache.org/thread/7ln106qxyw8sp7ljq40hs2p1lb1gdwj5
> >
> >
> >
> >
> > On Fri, 28 Jul 2023 at 06:18, Alexey Leonov-Vendrovskiy <
> > vendrov...@gmail.com> wrote:
> >
> >>>
> >>> `SELECT * FROM (SELECT $rowtime, * FROM t);`
> >>> Am I right that it will show `$rowtime` in output ?
> >>
> >>
> >> Yes, all explicitly selected columns become a part of the result (and
> >> intermediate) schema, and hence propagate.
> >>
> >> On Thu, Jul 27, 2023 at 2:40 PM Alexey Leonov-Vendrovskiy <
> >> vendrov...@gmail.com> wrote:
> >>
> >>> Thank you, Timo, for starting this FLIP!
> >>>
> >>> I propose the following change:
> >>>
> >>> Remove the requirement that DESCRIBE need to show system columns.
> >>>
> >>>
> >>> Some concrete vendor specific catalog implementations might prefer this
> >>&

Re: [DISCUSS] FLIP-348: Support System Columns in SQL and Table API

2023-07-27 Thread Alexey Leonov-Vendrovskiy
>
> `SELECT * FROM (SELECT $rowtime, * FROM t);`
> Am I right that it will show `$rowtime` in output ?


Yes, all explicitly selected columns become a part of the result (and
intermediate) schema, and hence propagate.

On Thu, Jul 27, 2023 at 2:40 PM Alexey Leonov-Vendrovskiy <
vendrov...@gmail.com> wrote:

> Thank you, Timo, for starting this FLIP!
>
> I propose the following change:
>
> Remove the requirement that DESCRIBE need to show system columns.
>
>
> Some concrete vendor specific catalog implementations might prefer this
> approach.
> Usually the same system columns are available on all (or family) of
> tables, and it can be easily captured in the documentation.
>
> For example, BigQuery does exactly this: there, pseudo-columns do not show
> up in the table schema in any place, but can be accessed via reference.
>
> So I propose we:
> a) Either we say that DESCRIBE doesn't show system columns,
> b) Or leave this vendor-specific / or configurable via flag (if needed).
>
> Regards,
> Alexey
>
> On Thu, Jul 27, 2023 at 3:27 AM Sergey Nuyanzin 
> wrote:
>
>> Hi Timo,
>>
>> Thanks for the FLIP.
>> I also tend to think that Option 3 is better.
>>
>> I would be also interested in a question mentioned by Benchao Li.
>> And a similar question about nested queries like
>> `SELECT * FROM (SELECT $rowtime, * FROM t);`
>> Am I right that it will show `$rowtime` in output ?
>>
>>
>> On Thu, Jul 27, 2023 at 6:58 AM Benchao Li  wrote:
>>
>> > Hi Timo,
>> >
>> > Thanks for the FLIP, I also like the idea and option 3 sounds good to
>> me.
>> >
>> > I would like to discuss a case which is not mentioned in the current
>> FLIP.
>> > How are the "System column"s expressed in intermediate result, e.g.
>> Join?
>> > E.g. `SELECT * FROM t1 JOIN t2`, I guess it should not include "system
>> > columns" from t1 and t2 as you proposed, and for `SELECT t1.$rowtime, *
>> > FROM t1 JOIN t2`, it should also be valid.
>> > Then the question is how to you plan to implement the "system columns",
>> do
>> > we need to add it to `RelNode` level? Or we just need to do it in the
>> > parsing/validating phase?
>> > I'm not sure that Calcite's "system column" feature is fully ready for
>> this
>> > since the code about this part is imported from the earlier project
>> before
>> > it gets into Apache, and has not been considered much in the past
>> > development.
>> >
>> >
>> > Jing Ge  于2023年7月26日周三 00:01写道:
>> >
>> > > Hi Timo,
>> > >
>> > > Thanks for your proposal. It is a very pragmatic feature. Among all
>> > options
>> > > in the FLIP, option 3 is one I prefer too and I'd like to ask some
>> > > questions to understand your thoughts.
>> > >
>> > > 1. I did some research on pseudo columns, just out of curiosity, do
>> you
>> > > know why most SQL systems do not need any prefix with their pseudo
>> > column?
>> > > 2. Some platform providers will use ${variable_name} to define their
>> own
>> > > configurations and allow them to be embedded into SQL scripts. Will
>> there
>> > > be any conflict with option 3?
>> > >
>> > > Best regards,
>> > > Jing
>> > >
>> > > On Tue, Jul 25, 2023 at 7:00 PM Konstantin Knauf 
>> > > wrote:
>> > >
>> > > > Hi Timo,
>> > > >
>> > > > this makes sense to me. Option 3 seems reasonable, too.
>> > > >
>> > > > Cheers,
>> > > >
>> > > > Konstantin
>> > > >
>> > > > Am Di., 25. Juli 2023 um 12:53 Uhr schrieb Timo Walther <
>> > > > twal...@apache.org
>> > > > >:
>> > > >
>> > > > > Hi everyone,
>> > > > >
>> > > > > I would like to start a discussion about introducing the concept
>> of
>> > > > > "System Columns" in SQL and Table API.
>> > > > >
>> > > > > The subject sounds bigger than it actually is. Luckily, Flink SQL
>> > > > > already exposes the concept of metadata columns. And this
>> proposal is
>> > > > > just a slight adjustment for how metadata columns can be used as
>> > system
>> > > > > columns.
>> > > > >
>> > > > > The biggest problem of metadata columns currently is that a
>> catalog
>> > > > > implementation can't provide them by default because they would
>> > affect
>> > > > > `SELECT *` when adding another one.
>> > > > >
>> > > > > Looking forward to your feedback on FLIP-348:
>> > > > >
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-348%3A+Support+System+Columns+in+SQL+and+Table+API
>> > > > >
>> > > > > Thanks,
>> > > > > Timo
>> > > > >
>> > > >
>> > > >
>> > > > --
>> > > > https://twitter.com/snntrable
>> > > > https://github.com/knaufk
>> > > >
>> > >
>> >
>> >
>> > --
>> >
>> > Best,
>> > Benchao Li
>> >
>>
>>
>> --
>> Best regards,
>> Sergey
>>
>


Re: [DISCUSS] FLIP-348: Support System Columns in SQL and Table API

2023-07-27 Thread Alexey Leonov-Vendrovskiy
Thank you, Timo, for starting this FLIP!

I propose the following change:

Remove the requirement that DESCRIBE need to show system columns.


Some concrete vendor specific catalog implementations might prefer this
approach.
Usually the same system columns are available on all (or family) of tables,
and it can be easily captured in the documentation.

For example, BigQuery does exactly this: there, pseudo-columns do not show
up in the table schema in any place, but can be accessed via reference.

So I propose we:
a) Either we say that DESCRIBE doesn't show system columns,
b) Or leave this vendor-specific / or configurable via flag (if needed).

Regards,
Alexey

On Thu, Jul 27, 2023 at 3:27 AM Sergey Nuyanzin  wrote:

> Hi Timo,
>
> Thanks for the FLIP.
> I also tend to think that Option 3 is better.
>
> I would be also interested in a question mentioned by Benchao Li.
> And a similar question about nested queries like
> `SELECT * FROM (SELECT $rowtime, * FROM t);`
> Am I right that it will show `$rowtime` in output ?
>
>
> On Thu, Jul 27, 2023 at 6:58 AM Benchao Li  wrote:
>
> > Hi Timo,
> >
> > Thanks for the FLIP, I also like the idea and option 3 sounds good to me.
> >
> > I would like to discuss a case which is not mentioned in the current
> FLIP.
> > How are the "System column"s expressed in intermediate result, e.g. Join?
> > E.g. `SELECT * FROM t1 JOIN t2`, I guess it should not include "system
> > columns" from t1 and t2 as you proposed, and for `SELECT t1.$rowtime, *
> > FROM t1 JOIN t2`, it should also be valid.
> > Then the question is how to you plan to implement the "system columns",
> do
> > we need to add it to `RelNode` level? Or we just need to do it in the
> > parsing/validating phase?
> > I'm not sure that Calcite's "system column" feature is fully ready for
> this
> > since the code about this part is imported from the earlier project
> before
> > it gets into Apache, and has not been considered much in the past
> > development.
> >
> >
> > Jing Ge  于2023年7月26日周三 00:01写道:
> >
> > > Hi Timo,
> > >
> > > Thanks for your proposal. It is a very pragmatic feature. Among all
> > options
> > > in the FLIP, option 3 is one I prefer too and I'd like to ask some
> > > questions to understand your thoughts.
> > >
> > > 1. I did some research on pseudo columns, just out of curiosity, do you
> > > know why most SQL systems do not need any prefix with their pseudo
> > column?
> > > 2. Some platform providers will use ${variable_name} to define their
> own
> > > configurations and allow them to be embedded into SQL scripts. Will
> there
> > > be any conflict with option 3?
> > >
> > > Best regards,
> > > Jing
> > >
> > > On Tue, Jul 25, 2023 at 7:00 PM Konstantin Knauf 
> > > wrote:
> > >
> > > > Hi Timo,
> > > >
> > > > this makes sense to me. Option 3 seems reasonable, too.
> > > >
> > > > Cheers,
> > > >
> > > > Konstantin
> > > >
> > > > Am Di., 25. Juli 2023 um 12:53 Uhr schrieb Timo Walther <
> > > > twal...@apache.org
> > > > >:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > I would like to start a discussion about introducing the concept of
> > > > > "System Columns" in SQL and Table API.
> > > > >
> > > > > The subject sounds bigger than it actually is. Luckily, Flink SQL
> > > > > already exposes the concept of metadata columns. And this proposal
> is
> > > > > just a slight adjustment for how metadata columns can be used as
> > system
> > > > > columns.
> > > > >
> > > > > The biggest problem of metadata columns currently is that a catalog
> > > > > implementation can't provide them by default because they would
> > affect
> > > > > `SELECT *` when adding another one.
> > > > >
> > > > > Looking forward to your feedback on FLIP-348:
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-348%3A+Support+System+Columns+in+SQL+and+Table+API
> > > > >
> > > > > Thanks,
> > > > > Timo
> > > > >
> > > >
> > > >
> > > > --
> > > > https://twitter.com/snntrable
> > > > https://github.com/knaufk
> > > >
> > >
> >
> >
> > --
> >
> > Best,
> > Benchao Li
> >
>
>
> --
> Best regards,
> Sergey
>


Re: SQL Gateway and SQL Client

2022-11-01 Thread Alexey Leonov-Vendrovskiy
Thanks Shengkai.

>  We don't have any detailed plan about this. But I know Livy has a
similar feature.
How can we check with Livy on this?

Back to the SQL client: is this the master issue tracking enabling SQL
Client <-> Gateway connection?
https://issues.apache.org/jira/browse/FLINK-29751 (Migrate SQL Client Local
Mode to use sql gateway)
It currently shows as unassigned. Is this really the case and no one is
working on enabling the connection from SQL Client to SQL Gateway.
Is the gateway mode planned to be supported for SQL Client in 1.17?

Regards,
Alexey

On Thu, Oct 27, 2022 at 11:07 PM Shengkai Fang  wrote:

> Hi.
>
> > Is there a possibility for us to get engaged and at least introduce
> initial changes to support authentication/authorization?
>
> Yes. You can write a FLIP about the design and change. We can discuss this
> in the dev mail. If the FLIP passes, we can develop it together.
>
> > Another question about persistent Gateway: did you have any specific
> thoughts about it or some draft design?
>
> We don't have any detailed plan about this. But I know Livy has a similar
> feature.
>
> Best,
> Shengkai
>
>
> Alexey Leonov-Vendrovskiy  于2022年10月27日周四 15:12写道:
>
>> Apologies from the delayed response on my side.
>>
>>  I think the authentication module is not part of our plan in 1.17
>>> because of the busy work. I think we'll start the design at the end of the
>>> release-1.17.
>>
>>
>> Is there a possibility for us to get engaged and at least introduce
>> initial changes to support authentication/authorization? Specifically,
>> changes in the API and in SQL Client.
>>
>> We expect the following authentication flow:
>>
>> On the SQL gateway we want to be able to use a delegation token.
>> SQL client should be able to supply an API key.
>> The SQL Gateway *would not *be submitting jobs on behalf of the client.
>>
>> Ideally it would be nice to introduce some interfaces in the SQL Gateway
>> that would allow implementing custom authentication and authorization.
>>
>> Another question about persistent Gateway: did you have any specific
>> thoughts about it or some draft design?
>>
>> Thanks,
>> Alexey
>>
>>
>> On Fri, Oct 21, 2022 at 1:13 AM Shengkai Fang  wrote:
>>
>>> Sorry for the late response.
>>>
>>> In the next version(Flink 1.17), we plan to support the SQL Client to
>>> submit the statement to the Flink SQL Gateway. The FLINK-29486
>>> <https://issues.apache.org/jira/browse/FLINK-29486> is the first step
>>> to remove the usage of the `Parser` in the client side, which needs to read
>>> the table schema during the converting sql node to operation. I think the 
>>> authentication
>>> module is not part of our plan in 1.17 because of the busy work. I think
>>> we'll start the design at the end of the release-1.17.
>>> But could you share more details about the requirements of the
>>> authentication?
>>> - Do you use the kerberos or delegation token or password to do the
>>> authentication?
>>> - After the authentication, do you need the sql gateway to submit the
>>> job on behalf of the client?
>>> - ...
>>>
>>> For detailed implementation, I think Hive and Presto are good
>>> examples to dig in.  If you have some thoughts about the authentication 
>>> module,
>>> please let me know.
>>>
>>> Best,
>>> Shengkai
>>>
>>> Alexey Leonov-Vendrovskiy  于2022年10月19日周三 00:37写道:
>>>
>>>> Thank you for the response, Yuxia!
>>>>
>>>> Shengkai, I would like to learn more about nearest and a bit more
>>>> distant plans about development of the SQL Gateway and the SQL Client.
>>>> Do you have a description of the work planned or maybe can share
>>>> general thoughts about the Authentication module, or Persistent Gateway.
>>>> How can the authentication part be addressed on the SQL Client side?
>>>>
>>>> Regards,
>>>> -Alexey
>>>>
>>>>
>>>> On Wed, Oct 12, 2022 at 11:24 PM yuxia 
>>>> wrote:
>>>>
>>>>> > In what Flink’s release the connection from SQL Client to the
>>>>> Gateway is
>>>>> expected to be added?
>>>>> Flink 1.17
>>>>>
>>>>> > “Authentication module” (2) and “Persistent Gateway” (4) as
>>>>> possible future work. Were there any recent discussions on these
>>>>> subjects?
>>>&g

Re: SQL Gateway and SQL Client

2022-10-27 Thread Alexey Leonov-Vendrovskiy
Apologies from the delayed response on my side.

 I think the authentication module is not part of our plan in 1.17 because
> of the busy work. I think we'll start the design at the end of the
> release-1.17.


Is there a possibility for us to get engaged and at least introduce initial
changes to support authentication/authorization? Specifically, changes in
the API and in SQL Client.

We expect the following authentication flow:

On the SQL gateway we want to be able to use a delegation token.
SQL client should be able to supply an API key.
The SQL Gateway *would not *be submitting jobs on behalf of the client.

Ideally it would be nice to introduce some interfaces in the SQL Gateway
that would allow implementing custom authentication and authorization.

Another question about persistent Gateway: did you have any specific
thoughts about it or some draft design?

Thanks,
Alexey


On Fri, Oct 21, 2022 at 1:13 AM Shengkai Fang  wrote:

> Sorry for the late response.
>
> In the next version(Flink 1.17), we plan to support the SQL Client to
> submit the statement to the Flink SQL Gateway. The FLINK-29486
> <https://issues.apache.org/jira/browse/FLINK-29486> is the first step to
> remove the usage of the `Parser` in the client side, which needs to read
> the table schema during the converting sql node to operation. I think the 
> authentication
> module is not part of our plan in 1.17 because of the busy work. I think
> we'll start the design at the end of the release-1.17.
> But could you share more details about the requirements of the
> authentication?
> - Do you use the kerberos or delegation token or password to do the
> authentication?
> - After the authentication, do you need the sql gateway to submit the job
> on behalf of the client?
> - ...
>
> For detailed implementation, I think Hive and Presto are good examples to
> dig in.  If you have some thoughts about the authentication module,
> please let me know.
>
> Best,
> Shengkai
>
> Alexey Leonov-Vendrovskiy  于2022年10月19日周三 00:37写道:
>
>> Thank you for the response, Yuxia!
>>
>> Shengkai, I would like to learn more about nearest and a bit more distant
>> plans about development of the SQL Gateway and the SQL Client.
>> Do you have a description of the work planned or maybe can share general
>> thoughts about the Authentication module, or Persistent Gateway.
>> How can the authentication part be addressed on the SQL Client side?
>>
>> Regards,
>> -Alexey
>>
>>
>> On Wed, Oct 12, 2022 at 11:24 PM yuxia 
>> wrote:
>>
>>> > In what Flink’s release the connection from SQL Client to the Gateway
>>> is
>>> expected to be added?
>>> Flink 1.17
>>>
>>> > “Authentication module” (2) and “Persistent Gateway” (4) as
>>> possible future work. Were there any recent discussions on these
>>> subjects?
>>> No recent discussions on these subjects, but I think it'll come in Flink
>>> 1.17
>>>
>>> > Another related topic: are there ideas around making SQL Gateway a
>>> multi-tenant
>>> component?
>>> Yes.
>>>
>>> Shengkaiis the maintainer of SQL Client and SQL gateway, maybe he can
>>> provide more information.
>>>
>>>
>>>
>>> Best regards,
>>> Yuxia
>>>
>>> - 原始邮件 -
>>> 发件人: "Alexey Leonov-Vendrovskiy" 
>>> 收件人: "dev" 
>>> 发送时间: 星期四, 2022年 10 月 13日 下午 12:33:08
>>> 主题: SQL Gateway and SQL Client
>>>
>>> Hi all,
>>>
>>> I’m Alexey from Confluent. This is my first email in this discussion
>>> list.
>>> I’m rather new to Flink, and to local customs of communication. I want to
>>> dive deeper and hopefully get more involved over time.
>>>
>>> Currently I have a few questions around SQL Gateway and SQL Client.
>>> Specifically I wanted to learn what is the vision around the nearest
>>> future
>>> of these two components.
>>>
>>> In what Flink’s release the connection from SQL Client to the Gateway is
>>> expected to be added? I was looking at
>>> https://issues.apache.org/jira/browse/FLINK-29486, and recently it got
>>> renamed from “Enable SQL Client to Connect SQL Gateway in Remote Mode” to
>>> “Introduce Client Parser to get statement type”.  I did some search, but
>>> didn’t find a good place where the client's work in this direction is
>>> discussed or tracked.
>>>
>>> A couple questions about the SQL Gateway. The FLIP-91
>>> <
>>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-91%3A+Support+SQL+Gateway#FLIP91:SupportSQLGateway-Futurework
>>> >
>>> mentions “Authentication module” (2) and “Persistent Gateway” (4) as
>>> possible future work. Were there any recent discussions on these
>>> subjects?
>>> Or maybe there are some ideas how to move these directions forward?
>>> Another
>>> related topic: are there ideas around making SQL Gateway a multi-tenant
>>> component?
>>>
>>> Thank you,
>>>
>>> Alexey
>>>
>>


Re: SQL Gateway and SQL Client

2022-10-18 Thread Alexey Leonov-Vendrovskiy
Thank you for the response, Yuxia!

Shengkai, I would like to learn more about nearest and a bit more distant
plans about development of the SQL Gateway and the SQL Client.
Do you have a description of the work planned or maybe can share general
thoughts about the Authentication module, or Persistent Gateway.
How can the authentication part be addressed on the SQL Client side?

Regards,
-Alexey


On Wed, Oct 12, 2022 at 11:24 PM yuxia  wrote:

> > In what Flink’s release the connection from SQL Client to the Gateway is
> expected to be added?
> Flink 1.17
>
> > “Authentication module” (2) and “Persistent Gateway” (4) as
> possible future work. Were there any recent discussions on these subjects?
> No recent discussions on these subjects, but I think it'll come in Flink
> 1.17
>
> > Another related topic: are there ideas around making SQL Gateway a
> multi-tenant
> component?
> Yes.
>
> Shengkaiis the maintainer of SQL Client and SQL gateway, maybe he can
> provide more information.
>
>
>
> Best regards,
> Yuxia
>
> - 原始邮件 -
> 发件人: "Alexey Leonov-Vendrovskiy" 
> 收件人: "dev" 
> 发送时间: 星期四, 2022年 10 月 13日 下午 12:33:08
> 主题: SQL Gateway and SQL Client
>
> Hi all,
>
> I’m Alexey from Confluent. This is my first email in this discussion list.
> I’m rather new to Flink, and to local customs of communication. I want to
> dive deeper and hopefully get more involved over time.
>
> Currently I have a few questions around SQL Gateway and SQL Client.
> Specifically I wanted to learn what is the vision around the nearest future
> of these two components.
>
> In what Flink’s release the connection from SQL Client to the Gateway is
> expected to be added? I was looking at
> https://issues.apache.org/jira/browse/FLINK-29486, and recently it got
> renamed from “Enable SQL Client to Connect SQL Gateway in Remote Mode” to
> “Introduce Client Parser to get statement type”.  I did some search, but
> didn’t find a good place where the client's work in this direction is
> discussed or tracked.
>
> A couple questions about the SQL Gateway. The FLIP-91
> <
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-91%3A+Support+SQL+Gateway#FLIP91:SupportSQLGateway-Futurework
> >
> mentions “Authentication module” (2) and “Persistent Gateway” (4) as
> possible future work. Were there any recent discussions on these subjects?
> Or maybe there are some ideas how to move these directions forward? Another
> related topic: are there ideas around making SQL Gateway a multi-tenant
> component?
>
> Thank you,
>
> Alexey
>


SQL Gateway and SQL Client

2022-10-12 Thread Alexey Leonov-Vendrovskiy
Hi all,

I’m Alexey from Confluent. This is my first email in this discussion list.
I’m rather new to Flink, and to local customs of communication. I want to
dive deeper and hopefully get more involved over time.

Currently I have a few questions around SQL Gateway and SQL Client.
Specifically I wanted to learn what is the vision around the nearest future
of these two components.

In what Flink’s release the connection from SQL Client to the Gateway is
expected to be added? I was looking at
https://issues.apache.org/jira/browse/FLINK-29486, and recently it got
renamed from “Enable SQL Client to Connect SQL Gateway in Remote Mode” to
“Introduce Client Parser to get statement type”.  I did some search, but
didn’t find a good place where the client's work in this direction is
discussed or tracked.

A couple questions about the SQL Gateway. The FLIP-91

mentions “Authentication module” (2) and “Persistent Gateway” (4) as
possible future work. Were there any recent discussions on these subjects?
Or maybe there are some ideas how to move these directions forward? Another
related topic: are there ideas around making SQL Gateway a multi-tenant
component?

Thank you,

Alexey