[jira] [Created] (FLINK-37308) HybridSourceReader doesn't support pauseOrResumeSplits

2025-02-11 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-37308: --- Summary: HybridSourceReader doesn't support pauseOrResumeSplits Key: FLINK-37308 URL: https://issues.apache.org/jira/browse/FLINK-37308 Project: Flink

Re: [VOTE] FLIP-497: Early Fire Support for Flink SQL Interval Join

2025-01-29 Thread Xingcan Cui
+1 (binding) Best, Xingcan On Mon, Jan 27, 2025 at 8:50 PM Venkatakrishnan Sowrirajan wrote: > +1 (non-binding) > > Regards > Venkata krishnan > > > On Mon, Jan 27, 2025 at 2:05 PM Weiqing Yang > wrote: > > > Hi All, > > > > I'd like to start a vote on FLIP-497: Early Fire Support for Flink SQ

Re: Re: Re: Re: [DISCUSS] Early Fire Support for Flink SQL Interval Join

2025-01-25 Thread Xingcan Cui
> > firing > > > > > > > available > > > > > > > >> >to users: `table.exec.emit.early-fire.en**abled` and > > > > > > > >> >`table.exec.emit.early-fire.de < > > > > > > > > > > > > > > > > > > > > > https://urldefense.com/v3/__http://table.exec.emit.early-fire.de_

Re: Plans for JDBC connector for 1.20?

2025-01-06 Thread Xingcan Cui
Hi Hang, Just want to follow up on this. What's the current progress? Are there any unassigned tickets we can help with? Best, Xingcan On Thu, Dec 12, 2024 at 9:46 PM Hang Ruan wrote: > Thanks, David & Peter. > > I would love to be the RM for jdbc-3.3.0. And the jdbc-3.3.0 should support > Fli

Re: [DISCUSS] Early Fire Support for Flink SQL Interval Join

2025-01-06 Thread Xingcan Cui
Hi Weiqing, Thanks for the proposal. IMO, adding early fire for time interval outer joins is feasible overall. I have a few questions. 1. The current output stream of a time interval outer join is an append-only stream. This change will make it a potential retractable stream. I'm not sure if the

[jira] [Created] (FLINK-36914) Sources with watermark alignment should wait for watermark generation

2024-12-16 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-36914: --- Summary: Sources with watermark alignment should wait for watermark generation Key: FLINK-36914 URL: https://issues.apache.org/jira/browse/FLINK-36914 Project: Flink

[jira] [Created] (FLINK-36893) Support periodic full checkpoint

2024-12-11 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-36893: --- Summary: Support periodic full checkpoint Key: FLINK-36893 URL: https://issues.apache.org/jira/browse/FLINK-36893 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-36004) Flink SQL returns wrong results for Paimon tables with complex schemas

2024-08-07 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-36004: --- Summary: Flink SQL returns wrong results for Paimon tables with complex schemas Key: FLINK-36004 URL: https://issues.apache.org/jira/browse/FLINK-36004 Project: Flink

[jira] [Created] (FLINK-35935) CREATE TABLE AS doesn't work with LIMIT

2024-07-30 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-35935: --- Summary: CREATE TABLE AS doesn't work with LIMIT Key: FLINK-35935 URL: https://issues.apache.org/jira/browse/FLINK-35935 Project: Flink Issue Type

[jira] [Created] (FLINK-35486) Potential sql expression generation issues on SQL gateway

2024-05-29 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-35486: --- Summary: Potential sql expression generation issues on SQL gateway Key: FLINK-35486 URL: https://issues.apache.org/jira/browse/FLINK-35486 Project: Flink

[jira] [Created] (FLINK-35485) JobMaster failed with "the job xx has not been finished"

2024-05-29 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-35485: --- Summary: JobMaster failed with "the job xx has not been finished" Key: FLINK-35485 URL: https://issues.apache.org/jira/browse/FLINK-35485 Project: Flink

[jira] [Created] (FLINK-34926) Adaptive auto parallelism doesn't work for a query

2024-03-24 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-34926: --- Summary: Adaptive auto parallelism doesn't work for a query Key: FLINK-34926 URL: https://issues.apache.org/jira/browse/FLINK-34926 Project: Flink Issue

[jira] [Created] (FLINK-34723) Parquet writer should restrict map keys to be not null

2024-03-18 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-34723: --- Summary: Parquet writer should restrict map keys to be not null Key: FLINK-34723 URL: https://issues.apache.org/jira/browse/FLINK-34723 Project: Flink Issue

[jira] [Created] (FLINK-34633) Support unnesting array constants

2024-03-08 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-34633: --- Summary: Support unnesting array constants Key: FLINK-34633 URL: https://issues.apache.org/jira/browse/FLINK-34633 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-34583) Bug for dynamic table option hints with multiple CTEs

2024-03-05 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-34583: --- Summary: Bug for dynamic table option hints with multiple CTEs Key: FLINK-34583 URL: https://issues.apache.org/jira/browse/FLINK-34583 Project: Flink Issue

Re: SQL return type change from 1.17 to 1.18

2023-12-07 Thread Xingcan Cui
ort example code. > > > > > > > > > > In a nutshell: > > > > > - Create a table with an 'ARRAY' column > > > > > - Run a select which returns this column > > > > > - The return type changes: > > > > >

[jira] [Created] (FLINK-33547) Primitive SQL array type after upgrading to Flink 1.18.0

2023-11-14 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-33547: --- Summary: Primitive SQL array type after upgrading to Flink 1.18.0 Key: FLINK-33547 URL: https://issues.apache.org/jira/browse/FLINK-33547 Project: Flink Issue

Re: Projection pushdown for Avro files seems to be buggy

2023-08-06 Thread Xingcan Cui
can > > After deep dive into the source code, I also think it is a bug. > > Best, > Ron > > Xingcan Cui 于2023年8月5日周六 23:27写道: > > > Hi all, > > > > We tried to read some Avro files with the Flink SQL (1.16.1) and noticed > > that the projection pushdown

Projection pushdown for Avro files seems to be buggy

2023-08-05 Thread Xingcan Cui
Hi all, We tried to read some Avro files with the Flink SQL (1.16.1) and noticed that the projection pushdown seems to be buggy. The Avro schema we used has 4 fields, namely f1, f2, f3 and f4. When using "SELECT *" or SELECT the first n fields (e.g., SELECT f1 or SELECT f1, f2) to read the table,

[jira] [Created] (FLINK-32171) Add PostStart hook to flink k8s operator helm

2023-05-23 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-32171: --- Summary: Add PostStart hook to flink k8s operator helm Key: FLINK-32171 URL: https://issues.apache.org/jira/browse/FLINK-32171 Project: Flink Issue Type: New

Re: Large schemas lead to long DataStream-to-table transformation names

2023-03-06 Thread Xingcan Cui
Oh, I just realized that FLIP-195 has already solved this. We'll upgrade our Flink version to 1.15+. Thanks! On Mon, Mar 6, 2023 at 10:08 AM Xingcan Cui wrote: > Hi Jark, > > Yes. I believe field names of the table would be enough to describe the > conversion operator. I'

Re: Large schemas lead to long DataStream-to-table transformation names

2023-03-06 Thread Xingcan Cui
> Normal table scan generates descriptions using field names instead of the > full schema. > Will that help in your case? > > Best, > Jark > > On Sat, 4 Mar 2023 at 06:57, Xingcan Cui wrote: > > > Hi all, > > > > We are dealing wit

Large schemas lead to long DataStream-to-table transformation names

2023-03-03 Thread Xingcan Cui
Hi all, We are dealing with some streams with large (nested) schemas. When using `t ableEnv.createTemporaryView()` to register a DataStream to a table, the transformation always gets a large name. It's not a big problem, but quite annoying since the UI and logs are hard to read. Internally, `Exte

[jira] [Created] (FLINK-31021) JavaCodeSplitter doesn't split static method properly

2023-02-10 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-31021: --- Summary: JavaCodeSplitter doesn't split static method properly Key: FLINK-31021 URL: https://issues.apache.org/jira/browse/FLINK-31021 Project: Flink

Re: Argo CD health check for FlinkDeployment

2022-11-16 Thread Xingcan Cui
atus) did not change significantly since the initial stable > release (1.0.0) in the last couple of months. > The jobStatus is also one thing that did not change at all. > > Cheers, > Gyula > > On Wed, Nov 16, 2022 at 4:21 PM Xingcan Cui wrote: > > > Hi all, > > >

Argo CD health check for FlinkDeployment

2022-11-16 Thread Xingcan Cui
Hi all, We are exploring Argo CD to manage `FlinkDeployment` resources but noticed that the health checking for it doesn't work properly. To give you some context, Argo CD uses Lua scripts to check some state-related fields and map them to three status values: "Healthy", "Progressing" and "Degrad

Re: Support decimal types with larger precisions

2021-08-30 Thread Xingcan Cui
kes sense to support this use case, but this definitely > needs > > a lot of work, > > and we need more investigation and discussion (maybe a new type?) > > > > Best, > > Jark > > > > > > On Mon, 30 Aug 2021 at 23:32, Xingcan Cui wrote: > > > &

Re: Support decimal types with larger precisions

2021-08-30 Thread Xingcan Cui
g pipelines and we > would need to offer a flag to bring back the old behavior. It would > definitely lead to a lot of testing work to not cause inconsistencies. > > Do you think this is a hard blocker for users? > > Regards, > Timo > > > On 28.08.21 00:21, Xingcan

Support decimal types with larger precisions

2021-08-27 Thread Xingcan Cui
Hi all, Recently, I was trying to load some CDC data from Oracle/Postgres databases and found that the current precision range [1, 38] for DecimalType may not meet the requirement for some source types. For instance, in Oracle, if a column is declared as `NUMBER` without precision and scale, the v

[jira] [Created] (FLINK-24007) Support Avro timestamp conversion with precision greater than three

2021-08-26 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-24007: --- Summary: Support Avro timestamp conversion with precision greater than three Key: FLINK-24007 URL: https://issues.apache.org/jira/browse/FLINK-24007 Project: Flink

Re: [VOTE] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-28 Thread Xingcan Cui
+1 (binding) Thanks, Xingcan On Thu, Sep 24, 2020 at 4:52 AM Jark Wu wrote: > +1 (binding) > > Best, > Jark > > On Thu, 24 Sep 2020 at 16:22, Jingsong Li wrote: > > > +1 (binding) > > > > Best, > > Jingsong > > > > On Thu, Sep 24, 2020 at 4:18 PM Kurt Young wrote: > > > > > +1 (binding) > > >

Re: [ANNOUNCE] New PMC member: Piotr Nowojski

2020-07-08 Thread Xingcan Cui
Congratulations, Piotr! Best, Xingcan On Wed, Jul 8, 2020, 21:53 Yang Wang wrote: > Congratulations Piotr! > > > Best, > Yang > > Dan Zou 于2020年7月8日周三 下午10:36写道: > > > Congratulations! > > > > Best, > > Dan Zou > > > > > 2020年7月8日 下午5:25,godfrey he 写道: > > > > > > Congratulations > > > > >

Re: FLIP 87: Primary key constraints in Table API

2019-11-23 Thread Xingcan Cui
Thanks for driving this, Dawid. I’m +1 on it. One minor suggestion: I think it’s better to override the `equals()` and `hashCode()` methods for `KeyConstraint`. Thanks, Xingcan > On Nov 23, 2019, at 2:40 AM, Jingsong Li wrote: > > +1 thanks dawid for driving this. > > Best > Jingsong Lee >

Re: [ANNOUNCE] Becket Qin joins the Flink PMC

2019-10-28 Thread Xingcan Cui
Congratulations, Becket! Best, Xingcan > On Oct 28, 2019, at 1:23 PM, Xuefu Z wrote: > > Congratulations, Becket! > > On Mon, Oct 28, 2019 at 10:08 AM Zhu Zhu wrote: > >> Congratulations Becket! >> >> Thanks, >> Zhu Zhu >> >> Peter Huang 于2019年10月29日周二 上午1:01写道: >> >>> Congratulations Be

[jira] [Created] (FLINK-13849) The back-pressure monitoring tab in Web UI may cause errors

2019-08-25 Thread Xingcan Cui (Jira)
Xingcan Cui created FLINK-13849: --- Summary: The back-pressure monitoring tab in Web UI may cause errors Key: FLINK-13849 URL: https://issues.apache.org/jira/browse/FLINK-13849 Project: Flink

Re: [DISCUSS] Enhance Support for Multicast Communication Pattern

2019-08-24 Thread Xingcan Cui
Hi all, Sorry for joining this thread late. Basically, I think enabling multicast pattern could be the right direction, but more detailed implementation policies need to be discussed. Two years ago, I filed an issue [1] about the multicast API. However, due to some reasons, it was laid aside.

Re: [ANNOUNCE] Jiangjie (Becket) Qin has been added as a committer to the Flink project

2019-07-18 Thread Xingcan Cui
Congrats Becket! Best, Xingcan On Thu, Jul 18, 2019, 07:17 Dian Fu wrote: > Congrats Becket! > > > 在 2019年7月18日,下午6:42,Danny Chan 写道: > > > >> Congratulations! > > > > Best, > > Danny Chan > > 在 2019年7月18日 +0800 PM6:29,Haibo Sun ,写道: > >> Congratulations Becket!Best, > >> Haibo > >> 在 2019-07-

Re: Support InMemoryTableSink or support retraction in CsvTableSink

2019-07-15 Thread Xingcan Cui
Hi Fan, It’s not hard to implement an “InMemoryTableSink” for debugging use by yourself. The main part should be a SinkFunction that deals with Tuple records. You can either show the changelog (the record itself) or print the materialized result (e.g., a result list) for each received item.

Re: [ANNOUNCE] Rong Rong becomes a Flink committer

2019-07-11 Thread Xingcan Cui
Congrats Rong! Best, Xingcan > On Jul 11, 2019, at 1:08 PM, Shuyi Chen wrote: > > Congratulations, Rong! > > On Thu, Jul 11, 2019 at 8:26 AM Yu Li > wrote: > Congratulations Rong! > > Best Regards, > Yu > > > On Thu, 11 Jul 2019 at 22:54, zhijiang

Re: [VOTE] How to Deal with Split/Select in DataStream API

2019-07-08 Thread Xingcan Cui
er option. > > Aljoscha > >> On 8. Jul 2019, at 06:39, Xingcan Cui > <mailto:xingc...@gmail.com>> wrote: >> >> Hi all, >> >> Thanks for your participation. >> >> In this thread, we got one +1 for option 1 and option 3, respective

Re: [VOTE] How to Deal with Split/Select in DataStream API

2019-07-07 Thread Xingcan Cui
fields, which is needed only for > splitting, in the outputs of flatMap functions. Replacing it with outputTags > would simplify data structures. > > Xingcan Cui mailto:xingc...@gmail.com>> 于 2019年7月5日周五 > 上午2:20写道: > Hi folks, > > Two weeks ago, I started a thread [

[VOTE] How to Deal with Split/Select in DataStream API

2019-07-04 Thread Xingcan Cui
Hi folks, Two weeks ago, I started a thread [1] discussing whether we should discard the split/select methods (which have been marked as deprecation since v1.7) in DataStream API. The fact is, these methods will cause "unexpected" results when using consecutively (e.g., ds.split(a).select(b).spli

Re: [ANNOUNCE] Jincheng Sun is now part of the Flink PMC

2019-06-24 Thread Xingcan Cui
Congratulations Jincheng and thanks for all you’ve done! Cheers, Xingcan > On Jun 25, 2019, at 1:59 AM, Tzu-Li (Gordon) Tai wrote: > > Congratulations Jincheng, great to have you on board :) > > Cheers, > Gordon > > On Tue, Jun 25, 2019, 11:31 AM Terry Wang wrote: > >> Congratulations Jinch

Re: About Deprecating split/select for DataStream API

2019-06-18 Thread Xingcan Cui
ate >>>>>> chains of split/select, which leads to undefined behavior. Take >>>>>> this >>>> for >>>>>> example: ds.split().select("a", "b").select("c", "d"). Which >>>>>> tags >>

About Deprecating split/select for DataStream API

2019-06-15 Thread Xingcan Cui
Hi all, Recently, I noticed that the split/select methods in DataStream API have been marked as deprecated since 1.7.2 and 1.8.0 (the related JIRA issue FLINK-11084 ). Although the two methods can be replaced by the more powerful side output f

[jira] [Created] (FLINK-12116) Args autocast will cause exception for plan transformation in TableAPI

2019-04-04 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-12116: --- Summary: Args autocast will cause exception for plan transformation in TableAPI Key: FLINK-12116 URL: https://issues.apache.org/jira/browse/FLINK-12116 Project: Flink

[jira] [Created] (FLINK-11769) The estimateDataTypesSize method in FlinkRelNode causes NPE for Multiset

2019-02-26 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-11769: --- Summary: The estimateDataTypesSize method in FlinkRelNode causes NPE for Multiset Key: FLINK-11769 URL: https://issues.apache.org/jira/browse/FLINK-11769 Project

Re: [DISCUSS] Contributing Chinese website and docs to Apache Flink

2019-02-12 Thread Xingcan Cui
Hi, I agree with the proposal, Gordon and Jark, and I think it's a good solution for major doc changes. We also created separated JIRAs for English documentation in the past. For minor doc changes, I think it’s better to encourage Chinese-speaking contributors to participate in the reviewing p

Re: [DISCUSS] Contributing Chinese website and docs to Apache Flink

2019-02-11 Thread Xingcan Cui
Hi, Thank you for working on this. I think assigning pages to someones and synchronizing them regularly may be a better idea. 1. We encourage contributors who are familiar with Chinese and would like to do the translation to provide Chinese doc updates in sync, but should not add extra burden

[jira] [Created] (FLINK-11227) The DescriptorProperties contains some bounds checking errors

2018-12-27 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-11227: --- Summary: The DescriptorProperties contains some bounds checking errors Key: FLINK-11227 URL: https://issues.apache.org/jira/browse/FLINK-11227 Project: Flink

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-11-30 Thread Xingcan Cui
gt;>>>>>>>>> To summarize, you propose a new method Table.cache(): Table >> that >>>>>>> will >>>>>>>>>>>>> trigger a job and write the result into some temporary storage >>>>> as >>>>>>>>>> defi

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-11-22 Thread Xingcan Cui
Hi all, @Shaoxuan, I think the lifecycle or access domain are both orthogonal to the cache problem. Essentially, this may be the first time we plan to introduce another storage mechanism other than the state. Maybe it’s better to first draw a big picture and then concentrate on a specific part?

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-11-21 Thread Xingcan Cui
so the > management of life cycle need to be discussed. > > Kind Regards > Xpray > > > > Xingcan Cui 于2018年11月21日周三 上午1:10写道: > >> Hi Becket, >> >> Thanks for bringing this up! For a long time, the intermediate cache >> problem has always been a

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-11-20 Thread Xingcan Cui
Hi Becket, Thanks for bringing this up! For a long time, the intermediate cache problem has always been a pain point of the Flink streaming model. As far as I know, it’s quite a block for iterate operations in batch-related libs such as Gelly and FlinkML. Actually, there’s an old JIRA[1], aim

[jira] [Created] (FLINK-10684) Improve the CSV reading process

2018-10-25 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10684: --- Summary: Improve the CSV reading process Key: FLINK-10684 URL: https://issues.apache.org/jira/browse/FLINK-10684 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-10463) Null literal cannot be properly parsed in Java Table API function call

2018-09-29 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10463: --- Summary: Null literal cannot be properly parsed in Java Table API function call Key: FLINK-10463 URL: https://issues.apache.org/jira/browse/FLINK-10463 Project: Flink

[jira] [Created] (FLINK-10323) A single backslash cannot be successfully parsed in Java Table API

2018-09-12 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10323: --- Summary: A single backslash cannot be successfully parsed in Java Table API Key: FLINK-10323 URL: https://issues.apache.org/jira/browse/FLINK-10323 Project: Flink

Re: [ANNOUNCE] New committer Gary Yao

2018-09-07 Thread Xingcan Cui
Congratulations, Gary! Xingcan > On Sep 7, 2018, at 11:20 PM, Hequn Cheng wrote: > > Congratulations Gary! > > Hequn > > On Fri, Sep 7, 2018 at 11:16 PM Matthias J. Sax wrote: > >> Congrats! >> >> On 09/07/2018 08:15 AM, Timo Walther wrote: >>> Congratulations, Gary! >>> >>> Timo >>> >>>

[jira] [Created] (FLINK-10201) The batchTestUtil was mistakenly used in some stream sql tests

2018-08-22 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10201: --- Summary: The batchTestUtil was mistakenly used in some stream sql tests Key: FLINK-10201 URL: https://issues.apache.org/jira/browse/FLINK-10201 Project: Flink

Re: [DISCUSS][TABLE] How to handle empty delete for UpsertSource

2018-08-21 Thread Xingcan Cui
Hi Hequn, Thanks for this discussion. Personally, I’m also in favor of option 3. There are two reasons for that: (1) A proctime-based upsert table source does not guarantee the records’ order, which means empty delete messages may not really be "empty". Simply discarding them may cause semanti

[jira] [Created] (FLINK-10108) DATE_FORMAT function in sql test throws a NumberFormatException

2018-08-09 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10108: --- Summary: DATE_FORMAT function in sql test throws a NumberFormatException Key: FLINK-10108 URL: https://issues.apache.org/jira/browse/FLINK-10108 Project: Flink

[jira] [Created] (FLINK-10049) Unify the processing logic for NULL arguments in SQL built-in functions

2018-08-03 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10049: --- Summary: Unify the processing logic for NULL arguments in SQL built-in functions Key: FLINK-10049 URL: https://issues.apache.org/jira/browse/FLINK-10049 Project: Flink

[jira] [Created] (FLINK-10014) Fix the decimal literal parameter problem for arithmetic functions in Table

2018-08-01 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10014: --- Summary: Fix the decimal literal parameter problem for arithmetic functions in Table Key: FLINK-10014 URL: https://issues.apache.org/jira/browse/FLINK-10014 Project

[jira] [Created] (FLINK-10009) Fix the casting problem for function TIMESTAMPADD in Table

2018-07-31 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10009: --- Summary: Fix the casting problem for function TIMESTAMPADD in Table Key: FLINK-10009 URL: https://issues.apache.org/jira/browse/FLINK-10009 Project: Flink

[jira] [Created] (FLINK-10008) Improve the LOG function in Table to support bases less than 1

2018-07-31 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-10008: --- Summary: Improve the LOG function in Table to support bases less than 1 Key: FLINK-10008 URL: https://issues.apache.org/jira/browse/FLINK-10008 Project: Flink

[jira] [Created] (FLINK-9977) Refine the docs for Table/SQL built-in functions

2018-07-26 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-9977: -- Summary: Refine the docs for Table/SQL built-in functions Key: FLINK-9977 URL: https://issues.apache.org/jira/browse/FLINK-9977 Project: Flink Issue Type

Re: Flink Query Optimizer

2018-07-13 Thread Xingcan Cui
Hi Albert, Calcite provides a rule-based optimizer (as a framework), which means users can customize it by adding rules. That’s exactly what Flink did. From the logical plan to the physical plan, the translations are triggered by different sets of rules, according to which the relational expres

Re: [DISCUSS] Long-term goal of making flink-table Scala-free

2018-07-02 Thread Xingcan Cui
Hi all, I also think about this problem these days and here are my thoughts. 1) We must admit that it’s really a tough task to interoperate with Java and Scala. E.g., they have different collection types (Scala collections v.s. java.util.*) and in Java, it's hard to implement a method which tak

Re: Flink table api

2018-07-02 Thread Xingcan Cui
Hi Amol, The “dynamic table” is just a logical concept, following which the Flink table API is designed. That means you don’t need to implement dynamic tables yourself. Flink table API provides different kinds of stream to stream joins in recent versions (from 1.4). The related docs can be foun

Re: Field Grouping in Flink

2018-05-17 Thread Xingcan Cui
Hi Garvit, you can use the `keyBy()` method[1] to partition a stream like the field grouping in Storm. Best, Xingcan [1] https://ci.apache.org/projects/flink/flink-docs-master/dev/stream/operators/#datastream-transformations > On May 17, 2018, at 4:04 PM, Garvit Sharma wrote: > > Hi, > > H

Re: [ANNOUNCE] Two new committers: Xingcan Cui and Nico Kruber

2018-05-09 Thread Xingcan Cui
gt;>>> Congratulations, Xingcan and Nico ! >>>> Nico is a good PR reviewer and I gained a lot from him. >>>> :)------ >>> 发件人:Fabian >>>> Hueske 发送时间:2018年5月9日(星期三) 02:53收件人:dev < >>>> dev@flink.a

[jira] [Created] (FLINK-8897) Rowtime materialization causes "mismatched type" AssertionError

2018-03-08 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-8897: -- Summary: Rowtime materialization causes "mismatched type" AssertionError Key: FLINK-8897 URL: https://issues.apache.org/jira/browse/FLINK-8897 Proj

[jira] [Created] (FLINK-8880) Validate configurations for SQL Client

2018-03-06 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-8880: -- Summary: Validate configurations for SQL Client Key: FLINK-8880 URL: https://issues.apache.org/jira/browse/FLINK-8880 Project: Flink Issue Type: Sub-task

Re: Verifying watermarks in integration test

2018-02-28 Thread Xingcan Cui
the > test approach there? > > Thanks, > Thomas > > > On Wed, Feb 21, 2018 at 12:43 AM, Xingcan Cui wrote: > >> Hi Thomas, >> >> some test cases in JoinHarnessTest <https://github.com/apache/ >> flink/blob/release-1.4/flink-libraries/flink-table

Re: Checkpointing Event Time Watermarks

2018-02-27 Thread Xingcan Cui
Hi Vijay, normally, maybe there’s no need to checkpoint the event times / watermarks since they are automatically generated based on the records. What’s your intention? Best, Xingcan > On 27 Feb 2018, at 8:50 PM, vijay kansal wrote: > > Hi All > > Is there a way to checkpoint event time wat

Re: [DISCUS] Flink SQL Client dependency management

2018-02-27 Thread Xingcan Cui
Hi Timo, thanks for your efforts. Personally, I think the second option would be better and here are my feelings. (1) The SQL client is designed to offer a convenient way for users to manipulate data with Flink. Obviously, the second option would be more easy-to-use. (2) The script will hel

Re: Verifying watermarks in integration test

2018-02-21 Thread Xingcan Cui
Hi Thomas, some test cases in JoinHarnessTest show how to verify the emitted watermarks. Hope this helps. Best, Xingcan > On 21 Feb 2018,

[jira] [Created] (FLINK-8630) To support JSON schema to TypeInformation conversion

2018-02-10 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-8630: -- Summary: To support JSON schema to TypeInformation conversion Key: FLINK-8630 URL: https://issues.apache.org/jira/browse/FLINK-8630 Project: Flink Issue Type

[jira] [Created] (FLINK-8407) Setting the parallelism after a partitioning operation should be forbidden

2018-01-10 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-8407: -- Summary: Setting the parallelism after a partitioning operation should be forbidden Key: FLINK-8407 URL: https://issues.apache.org/jira/browse/FLINK-8407 Project: Flink

[jira] [Created] (FLINK-8316) The CsvTableSink and the CsvInputFormat are not in sync

2017-12-23 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-8316: -- Summary: The CsvTableSink and the CsvInputFormat are not in sync Key: FLINK-8316 URL: https://issues.apache.org/jira/browse/FLINK-8316 Project: Flink Issue Type

[jira] [Created] (FLINK-8258) Enable query configuration for batch queries

2017-12-13 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-8258: -- Summary: Enable query configuration for batch queries Key: FLINK-8258 URL: https://issues.apache.org/jira/browse/FLINK-8258 Project: Flink Issue Type

[jira] [Created] (FLINK-8257) Unify the value checks for setParallelism()

2017-12-13 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-8257: -- Summary: Unify the value checks for setParallelism() Key: FLINK-8257 URL: https://issues.apache.org/jira/browse/FLINK-8257 Project: Flink Issue Type

Re: Test conflicts in JoinTest

2017-12-08 Thread Xingcan Cui
ll need to pay attention to it. Thanks, Xingcan On Fri, Dec 8, 2017 at 7:25 PM, Xingcan Cui wrote: > Hi all, > > Recently I'm trying to add some tests to > *org.apache.flink.table.api.stream.table.JoinTest*, but encountered a > strange problem. A test case could successfull

Test conflicts in JoinTest

2017-12-08 Thread Xingcan Cui
Hi all, Recently I'm trying to add some tests to *org.apache.flink.table.api.stream.table.JoinTest*, but encountered a strange problem. A test case could successfully run in an isolated mode, but failed (threw an *CannotPlanException* in *TableEnvironment.runVolcanoPlanner()*) when ran with anoth

[jira] [Created] (FLINK-8094) Support other types for ExistingField rowtime extractor

2017-11-16 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-8094: -- Summary: Support other types for ExistingField rowtime extractor Key: FLINK-8094 URL: https://issues.apache.org/jira/browse/FLINK-8094 Project: Flink Issue Type

Re: [ANNOUNCE] New committer: Haohui Mai

2017-11-01 Thread Xingcan Cui
Congratulations! On Wed, Nov 1, 2017 at 9:37 PM, Kurt Young wrote: > Congrats and welcome on board! > > Best, > Kurt > > On Wed, Nov 1, 2017 at 8:15 PM, Hai Zhou wrote: > >> Congratulations! >> >> On 1. Nov 2017, at 10:13, Shaoxuan Wang wrote: >> >> Congratulations! >> >> On Wed, Nov 1, 2017 a

[jira] [Created] (FLINK-7865) Remove predicate restrictions on TableFunction left outer join

2017-10-18 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-7865: -- Summary: Remove predicate restrictions on TableFunction left outer join Key: FLINK-7865 URL: https://issues.apache.org/jira/browse/FLINK-7865 Project: Flink

[jira] [Created] (FLINK-7854) Reject lateral table outer joins with predicates in SQL

2017-10-16 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-7854: -- Summary: Reject lateral table outer joins with predicates in SQL Key: FLINK-7854 URL: https://issues.apache.org/jira/browse/FLINK-7854 Project: Flink Issue Type

[jira] [Created] (FLINK-7853) Reject table function outer joins with predicates in Table API

2017-10-16 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-7853: -- Summary: Reject table function outer joins with predicates in Table API Key: FLINK-7853 URL: https://issues.apache.org/jira/browse/FLINK-7853 Project: Flink

Re: [DISCUSS] Table API / SQL internal timestamp handling

2017-07-31 Thread Xingcan Cui
amp depends on two fields and not just on > one." In the windowed join cases, we have to buffer all the delta data > between watermarks of two inputs. It is the user's responsibility (if > she/he wants to reduce the cost) to align watermarks of the stream sources > as much as

Re: [DISCUSS] Table API / SQL internal timestamp handling

2017-07-31 Thread Xingcan Cui
> this to timestamps that are actually used as such. > > > > I'm more curious about how can we define the watermark strategies in > order > > to make sure all timestamp columns are aligned to watermarks. Especially, > > when the watermark has been defined in the

Re: [DISCUSS] Table API / SQL internal timestamp handling

2017-07-27 Thread Xingcan Cui
> > > > > > > > > HUAWEI TECHNOLOGIES Duesseldorf GmbH > > > > German Research Center > > > > Munich Office > > > > Riesstrasse 25, 80992 München > > > > > > > > E-mail: radu.tudo...@huawei.com > > > > Mob

Re: [DISCUSS] Table API / SQL internal timestamp handling

2017-07-25 Thread Xingcan Cui
Hi all, @Fabian, thanks for raising this. @Radu and Jark, personally I think the timestamp field is critical for query processing and thus should be declared as (or supposed to be) NOT NULL. In addition, I think the event-time semantic of the join results should be automatically decided by the sy

[jira] [Created] (FLINK-7245) Enhance the operators to support holding back watermarks

2017-07-23 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-7245: -- Summary: Enhance the operators to support holding back watermarks Key: FLINK-7245 URL: https://issues.apache.org/jira/browse/FLINK-7245 Project: Flink Issue

[jira] [Created] (FLINK-6936) Add multiple targets support for custom partitioner

2017-06-16 Thread Xingcan Cui (JIRA)
Xingcan Cui created FLINK-6936: -- Summary: Add multiple targets support for custom partitioner Key: FLINK-6936 URL: https://issues.apache.org/jira/browse/FLINK-6936 Project: Flink Issue Type

Re: Proposal about inner join in Flink

2017-05-17 Thread Xingcan Cui
is closely associated with "Retract" (proposed in FLINK-6047). > I have not completely read through your doc, but the window-joins that you > mentioned are definitely the topics that we are also interested. Will read > it carefully and left comments on your doc. Thanks! > >

Proposal about inner join in Flink

2017-05-17 Thread Xingcan Cui
Hi everyone, Recently, I drafted a proposal about inner join in Flink ( http://goo.gl/4AdR7h). This document reviews some related work on the Table/SQL topic and it provides a relatively complete view about the inner join semantics and implementation. Besides, I also share my (objective) thoughts

Re: Sliding Window - Weird behaviour

2017-04-13 Thread Xingcan Cui
​Hi harish, I will not argue for the correctness of the result​s, but just tell you why this happens. The countWindow(2, 1) can be regarded as two separate processes: 1) maintain a window whose size *not exceeds* 2 and 2) trigger window evaluation every single record. Actually, in Flink the two

Re: Question about the process order in stream aggregate

2017-04-12 Thread Xingcan Cui
timestamp of "arrival", but it will > > produce sorted output as you "order by rowTime". Hope it helps. > > > > Best, > > Stefano > > > > -Original Message- > > From: Xingcan Cui [mailto:xingc...@gmail.com] > > Sent: Wednesday, Ap

Re: Question about the process order in stream aggregate

2017-04-11 Thread Xingcan Cui
you using parallelism 1 for the test? procTime semantics deals with > the objects as they loaded in the operators. It could be the co-occuring > partitioned events (in the same MS time frame) are processed in parallel > and then the output is produced in different order. > > I suggest y

  1   2   >