[DISCUSS] (FLINK-16648) Clean up the input parameter StreamQueryConfig of the API in StreamTableEnvironment

classic Classic list List threaded Threaded
4 messages Options
Reply | Threaded
Open this post in threaded view
|

[DISCUSS] (FLINK-16648) Clean up the input parameter StreamQueryConfig of the API in StreamTableEnvironment

jinhai wang
Hi Devs
 
I would like to start the formal discussion about FLINK-16648.

StreamQueryConfig is deprecated and replaced by TableConfig in the future, and TableEnvironment class has config object.

Do we need to delete the interfaces in StreamTableEnvironment and StreamTableEnvironmentImpl:
toAppendStream(Table table, Class<T> clazz, StreamQueryConfig queryConfig);
toAppendStream(Table table, TypeInformation<T> typeInfo, StreamQueryConfig queryConfig);
toRetractStream(Table table, Class<T> clazz, StreamQueryConfig queryConfig);
toRetractStream(Table table, TypeInformation<T> typeInfo, StreamQueryConfig queryConfig);
void sqlUpdate(String stmt, StreamQueryConfig config);
void insertInto(Table table, StreamQueryConfig queryConfig, String sinkPath, String... sinkPathContinued);


Best Regards

[hidden email]

Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] (FLINK-16648) Clean up the input parameter StreamQueryConfig of the API in StreamTableEnvironment

Jingsong Li
Hi Jinhai, thanks for driving.

+1 to remove, I think we can remove StreamQueryConfig too. since we have
deprecated StreamQueryConfig two versions.

Remember to record it in release notes of issue.

Best,
Jingsong Lee

On Wed, Mar 18, 2020 at 5:58 PM jinhai wang <[hidden email]> wrote:

> Hi Devs
>
> I would like to start the formal discussion about FLINK-16648.
>
> StreamQueryConfig is deprecated and replaced by TableConfig in the future,
> and TableEnvironment class has config object.
>
> Do we need to delete the interfaces in StreamTableEnvironment and
> StreamTableEnvironmentImpl:
> toAppendStream(Table table, Class<T> clazz, StreamQueryConfig queryConfig);
> toAppendStream(Table table, TypeInformation<T> typeInfo, StreamQueryConfig
> queryConfig);
> toRetractStream(Table table, Class<T> clazz, StreamQueryConfig
> queryConfig);
> toRetractStream(Table table, TypeInformation<T> typeInfo,
> StreamQueryConfig queryConfig);
> void sqlUpdate(String stmt, StreamQueryConfig config);
> void insertInto(Table table, StreamQueryConfig queryConfig, String
> sinkPath, String... sinkPathContinued);
>
>
> Best Regards
>
> [hidden email]
>
>

--
Best, Jingsong Lee
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] (FLINK-16648) Clean up the input parameter StreamQueryConfig of the API in StreamTableEnvironment

jinhai wang
Also, QueryConfig is deprecated,we can remove BatchTableEnvironment and BatchTableEnvironmentImpl api about BatchQueryConfig?
And remove QueryConfig interface finally


Best Regards

[hidden email]

> 2020年3月18日 下午6:05,Jingsong Li <[hidden email]> 写道:
>
> Hi Jinhai, thanks for driving.
>
> +1 to remove, I think we can remove StreamQueryConfig too. since we have
> deprecated StreamQueryConfig two versions.
>
> Remember to record it in release notes of issue.
>
> Best,
> Jingsong Lee
>
> On Wed, Mar 18, 2020 at 5:58 PM jinhai wang <[hidden email]> wrote:
>
>> Hi Devs
>>
>> I would like to start the formal discussion about FLINK-16648.
>>
>> StreamQueryConfig is deprecated and replaced by TableConfig in the future,
>> and TableEnvironment class has config object.
>>
>> Do we need to delete the interfaces in StreamTableEnvironment and
>> StreamTableEnvironmentImpl:
>> toAppendStream(Table table, Class<T> clazz, StreamQueryConfig queryConfig);
>> toAppendStream(Table table, TypeInformation<T> typeInfo, StreamQueryConfig
>> queryConfig);
>> toRetractStream(Table table, Class<T> clazz, StreamQueryConfig
>> queryConfig);
>> toRetractStream(Table table, TypeInformation<T> typeInfo,
>> StreamQueryConfig queryConfig);
>> void sqlUpdate(String stmt, StreamQueryConfig config);
>> void insertInto(Table table, StreamQueryConfig queryConfig, String
>> sinkPath, String... sinkPathContinued);
>>
>>
>> Best Regards
>>
>> [hidden email]
>>
>>
>
> --
> Best, Jingsong Lee

Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] (FLINK-16648) Clean up the input parameter StreamQueryConfig of the API in StreamTableEnvironment

Timo Walther-2
Hi Jinhai,

yes, we can remove this deprecated method. Actually, I started with it
but I stopped because we were missing a validation feature in
ConfigOptions that ensures that min and max retention time satisfies
some conditions (e.g. min is always smaller than max and some minimum
difference).

We can solve this by having a little config option validation utility at
a well-defined location close to the config option. The min/max value
should only be used after calling this utility.

Regards,
Timo


On 18.03.20 11:38, jinhai wang wrote:

> Also, QueryConfig is deprecated,we can remove BatchTableEnvironment and BatchTableEnvironmentImpl api about BatchQueryConfig?
> And remove QueryConfig interface finally
>
>
> Best Regards
>
> [hidden email]
>
>> 2020年3月18日 下午6:05,Jingsong Li <[hidden email]> 写道:
>>
>> Hi Jinhai, thanks for driving.
>>
>> +1 to remove, I think we can remove StreamQueryConfig too. since we have
>> deprecated StreamQueryConfig two versions.
>>
>> Remember to record it in release notes of issue.
>>
>> Best,
>> Jingsong Lee
>>
>> On Wed, Mar 18, 2020 at 5:58 PM jinhai wang <[hidden email]> wrote:
>>
>>> Hi Devs
>>>
>>> I would like to start the formal discussion about FLINK-16648.
>>>
>>> StreamQueryConfig is deprecated and replaced by TableConfig in the future,
>>> and TableEnvironment class has config object.
>>>
>>> Do we need to delete the interfaces in StreamTableEnvironment and
>>> StreamTableEnvironmentImpl:
>>> toAppendStream(Table table, Class<T> clazz, StreamQueryConfig queryConfig);
>>> toAppendStream(Table table, TypeInformation<T> typeInfo, StreamQueryConfig
>>> queryConfig);
>>> toRetractStream(Table table, Class<T> clazz, StreamQueryConfig
>>> queryConfig);
>>> toRetractStream(Table table, TypeInformation<T> typeInfo,
>>> StreamQueryConfig queryConfig);
>>> void sqlUpdate(String stmt, StreamQueryConfig config);
>>> void insertInto(Table table, StreamQueryConfig queryConfig, String
>>> sinkPath, String... sinkPathContinued);
>>>
>>>
>>> Best Regards
>>>
>>> [hidden email]
>>>
>>>
>>
>> --
>> Best, Jingsong Lee