Re: [Vote] FLIP-298: Unifying the Implementation of SlotManager
+1 (binding) Zhanghao Chen 于 2023年3月10日周五 下午5:07写道: > Thanks Weihua. +1 (non-binding) > > Best, > Zhanghao Chen > > From: Weihua Hu > Sent: Thursday, March 9, 2023 13:27 > To: dev > Subject: [Vote] FLIP-298: Unifying the Implementation of SlotManager > > Hi Everyone, > > I would like to start the vote on FLIP-298: Unifying the Implementation > of SlotManager [1]. The FLIP was discussed in this thread [2]. > > This FLIP aims to unify the implementation of SlotManager in > order to reduce maintenance costs. > > The vote will last for at least 72 hours (03/14, 15:00 UTC+8) > unless there is an objection or insufficient votes. Thank you all. > > [1] > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-298%3A+Unifying+the+Implementation+of+SlotManager > [2]https://lists.apache.org/thread/ocssfxglpc8z7cto3k8p44mrjxwr67r9 > > Best, > Weihua >
[jira] [Created] (FLINK-31403) CliClientITCase print unexpected border when printing explain results
Shengkai Fang created FLINK-31403: - Summary: CliClientITCase print unexpected border when printing explain results Key: FLINK-31403 URL: https://issues.apache.org/jira/browse/FLINK-31403 Project: Flink Issue Type: Bug Components: Table SQL / Client Affects Versions: 1.17.0, 1.18.0 Reporter: Shengkai Fang -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: [DISCUSS] FLIP-296: Watermark options for table API & SQL
Hi all, I have modified the name of idle-timeout option in the flip @Timo > You are right. That was just a copy paste mistake. It should be > `scan.watermark.idle-timeout`. If there are no questions, we will start the voting thread on 3/15(wednesday ). Best Kui Timo Walther 于2023年3月9日周四 17:11写道: > > I was wondering why you, exceptionally, suggested 'scan.idle-timeout' > instead of 'scan.watermark.idle-timeout'. I must miss something here. > > @Jing: You are right. That was just a copy paste mistake. It should be > `scan.watermark.idle-timeout`. > > @Kui: Can you fix that in the FLIP? Sorry, for this typo. > > > users might feel confused after using those OPTIONs, since they might > not be aware of what happens underneath > > I agree that those options will not make the life easier for most SQL > users. I would consider those options intended for power users. Usually, > the data platform team should come up with well-defined watermark > semantics before exposing tables to SQL users. > > Regards, > Timo > > > On 07.03.23 13:15, Kui Yuan wrote: > > Hi Jing, > > > > Thanks for your advice. In upcoming PR, I will explain all these things > > (Including flip-182,flip-217,etc.) clearly in the flink doc to make sure > > the user can understand the behavior behind it. > > > > Best, > > Kui > > > > Jing Ge 于2023年3月7日周二 19:42写道: > > > >> Hi Kui, > >> > >> Thanks for adding it into the Flip. There is one more thing wrt this > topic > >> you might want to pay attention to, a little bit off-topic, is that > Flink > >> SQL users might not be familiar with use cases of low level Datastream > API. > >> IMHO, it is highly recommended(mandatory) to write the dependency you > just > >> described in your last email and all related information in Flink doc > >> during developing this FLIP within the upcoming PR. Without those > >> guidelines in doc, users might feel confused after using those OPTIONs, > >> since they might not be aware of what happens underneath, and therefore > >> don't know why it does not work even if they did everything right at > Flink > >> SQL level. > >> > >> Best regards, > >> Jing > >> > >> > >> On Tue, Mar 7, 2023 at 6:36 AM Kui Yuan wrote: > >> > >>> Hi Jing, > >>> > >>> > >>> Thanks for the reminder. The aim of this flip is letting the sql users > to > >>> use those features in the Datastream API, we don't intend to extend > >>> flip-217. In my opinion, the watermark alignment with only one source > can > >>> be configured by the options given in flip, and if the source connector > >>> does not implement flip-217, the task will run with an error, reminding > >> the > >>> user to use `pipeline.watermark-alignment.allow- > >> unaligned-source-splits`, > >>> but maybe these behaviors are not understood by everyone, I will add > some > >>> tips about flip-217 in the flip to let users understand the behavior in > >> the > >>> case of source splits. > >>> > >>> > >>> Best, > >>> > >>> Kui Yuan > >>> > >>> Jing Ge 于2023年3月7日周二 04:23写道: > >>> > Hi Kui, > > Thanks for pointing that out. I knew FLIP-217 which was done by an > engineer working in my team. As far as I am concerned, your FLIP > >> should > answer the following questions: > > 1. How to enable the watermark alignment of a source splits with Flink > >>> SQL? > e.g. which options should be used if only one source is used? > > 2. Default behaviour. i.e. Flink SQL users should be aware that > >> watermark > alignment of source split will only work for sources that implement > FLIP-217 properly. Should users take care of > `pipeline.watermark-alignment.allow-unaligned-source-splits` > while using Flink SQL? > > Best regards, > Jing > > > On Fri, Mar 3, 2023 at 8:46 AM Kui Yuan wrote: > > > Hi all, > > > > Thanks for all. There are more questions and I will answer one by > >> one. > > > > @Jark Thanks for your tips. For the first question, I will add more > details > > in the flip, and give a POC[1] so that pepole can know how I'm > >>> currently > > implementing these features. > > > >> IIRC, this is the first time we introduce the framework-level > >>> connector > >> options that the option is not recognized and handled by > >> connectors. > >> The FLIP should cover how framework filters the watermark related > options > >> to avoid discover connector factory failed, and what happens if the > >> connector already supported the conflict options > > > > For the second question, We know that the default strategy is > 'on-periodic' > > in SQL layer, and the default interval is 200ms. The reason for > >> emiting > > watermark periodically is that the time advancement of consecutive > >>> events > > may be very small, we don't need to calculate watermark for each > >> event. > > Same for 'on-event' strategy, so my idea is that we can set a fixed >
[jira] [Created] (FLINK-31402) Remove ConverterUtils Deprecated Method
Shilun Fan created FLINK-31402: -- Summary: Remove ConverterUtils Deprecated Method Key: FLINK-31402 URL: https://issues.apache.org/jira/browse/FLINK-31402 Project: Flink Issue Type: Improvement Components: Deployment / YARN Affects Versions: 1.17.0 Reporter: Shilun Fan Many methods in ConverterUtils are obsolete, we can replace the obsolete methods and remove this class(ConverterUtils). -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (FLINK-31401) testTransformationSetParallelism fails on 10 core machines
Márton Balassi created FLINK-31401: -- Summary: testTransformationSetParallelism fails on 10 core machines Key: FLINK-31401 URL: https://issues.apache.org/jira/browse/FLINK-31401 Project: Flink Issue Type: Bug Components: API / DataStream, Tests Affects Versions: 1.17.0 Reporter: Márton Balassi Assignee: Márton Balassi StreamingJobGraphGenerator#testTransformationSetParallelism fails if it is run in an environment where the default parallelism is 10: {noformat} org.opentest4j.AssertionFailedError: expected: 3 but was: 2 Expected :3 Actual :2 {noformat} The fix is trivial, we need to make an implicit assumption in the test about paralellisms explicit. -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: [ANNOUNCE] Release 1.17.0, release candidate #1
Hi Martijn, Thanks a lot. Fix has be merged. Best regards, Yingjie Martijn Visser 于2023年3月10日周五 03:32写道: > Hi Yingjie, > > Thanks for the test and identifying the issue, this is super helpful! > > To all others, please continue your testing on this RC so that if there are > more blockers to be found, we can fix them with the next RC and have > (hopefully) a successful vote on it. > > Best regards, > > Martijn > > On Thu, Mar 9, 2023 at 4:54 PM Yingjie Cao > wrote: > > > Hi community and release managers: > > > > When testing the release candidate #1 for batch scenario, I found a > > potential deadlock issue of blocking shuffle. I have created a ticket [1] > > for it and marked it as blocker. I will fix it no later than tomorrow. > > > > [1] https://issues.apache.org/jira/browse/FLINK-31386 > > > > Best regards, > > Yingjie > > > > Qingsheng Ren 于2023年3月9日周四 13:51写道: > > > > > Hi everyone, > > > > > > The RC1 for Apache Flink 1.17.0 has been created. This RC currently is > > for > > > preview only to facilitate the integrated testing since the release > > > announcement is still under review. The voting process will be > triggered > > > once the announcement is ready. It has all the artifacts that we would > > > typically have for a release, except for the release note and the > website > > > pull request for the release announcement. > > > > > > The following contents are available for your review: > > > > > > - The preview source release and binary convenience releases [1], which > > > are signed with the key with fingerprint A1BD477F79D036D2C30C [2]. > > > - all artifacts that would normally be deployed to the Maven > > > Central Repository [3]. > > > - source code tag "release-1.17.0-rc1" [4] > > > > > > Your help testing the release will be greatly appreciated! And we'll > > > create the voting thread as soon as all the efforts are finished. > > > > > > [1] https://dist.apache.org/repos/dist/dev/flink/flink-1.17.0-rc1 > > > [2] https://dist.apache.org/repos/dist/release/flink/KEYS > > > [3] > > https://repository.apache.org/content/repositories/orgapacheflink-1591 > > > [4] https://github.com/apache/flink/releases/tag/release-1.17.0-rc1 > > > > > > Best regards, > > > Qingsheng, Leonard, Matthias and Martijn > > > > > >
Re: [DISCUSS] FLIP-298: Unifying the Implementation of SlotManager
+1 on the proposal. I'm wondering about the scale down behavior of the unified implementation. Does the new unified implementation prioritize releasing entire task managers in favor of evenly spreading out task managers? Consider a scale down from parallelism 15 to parallelism 10 where each task manager has 5 slots. We could either spread out 10 slots among the 3 task managers, or fit the 10 slots in 2 task managers and surrender the task manager (at least on k8s). From a cost saving perspective, the latter would be preferable. -Max On Thu, Mar 9, 2023 at 4:17 PM Matthias Pohl wrote: > > Thanks for your clarification. I have nothing else to add to the > discussion. +1 from my side to proceed > > On Wed, Mar 8, 2023 at 4:16 AM Weihua Hu wrote: > > > Thanks Yangze for your attention, this would be a great help. > > > > And thanks Matthias too. > > > > FLIP-156 [1] mentions some incompatibility between fine-grained resource > > > management and reactive mode. I assume that this is independent of the > > > SlotManager and replacing the DSM with the FGSM wouldn't affect reactive > > > mode? > > > > Yes. This incompatibility is independent of SlotManager. That means the > > AdpativeScheduler will always ignore the resource requirement set by > > slotSharingGroup and declare Unknown ResourceProfile to SlotManager. > > So, using FGSM as default will not affect reactive mode. > > > > About the heterogeneous TaskManager: This is a feature that's also not > > > supported in the DSM right now, is it? We should state that fact in the > > > FLIP if we mentioned that we don't want to implement it for the FSGM. > > > > Yes, both DSM and FGSM do not support request heterogeneous > > TaskManager right now. Heterogeneous will make the resource allocation > > logic more complicated, such as the resource deadlock if request A > > allocated > > the bigger slot B and then request B could not allocate the small slot A. > > We > > need to think more before starting to support the heterogeneous task > > manager. > > So, we don't want to implement heterogeneity in this FLIP. > > > > Best, > > Weihua > > > > > > On Wed, Mar 8, 2023 at 12:44 AM Matthias Pohl > > wrote: > > > > > Thanks for updating the FLIP and adding more context to it. Additionally, > > > thanks to Xintong and Yangze for offering your expertise here as > > > contributors to the initial FineGrainedSlotManager implementation. > > > > > > The remark on cutting out functionality was only based on some > > superficial > > > initial code reading. I cannot come up with a better code structure > > myself. > > > Therefore, I'm fine with not refactoring the code as part of this FLIP. > > > > > > The strategies that were proposed around making sure that the refactoring > > > is properly backed by tests sound reasonable. My initial concern was > > based > > > on the fact that we might have unit test scenarios for the DSM that are > > not > > > covered in the unit tests of the FSGM. In that case, swapping the DSM > > with > > > the FSGM might not be good enough. Going over the GSM tests to make sure > > > that we're not accidentally deleting test scenarios sounds good to me. > > > Thanks, Weihua. > > > > > > FLIP-156 [1] mentions some incompatibility between fine-grained resource > > > management and reactive mode. I assume that this is independent of the > > > SlotManager and replacing the DSM with the FGSM wouldn't affect reactive > > > mode? > > > > > > About the heterogeneous TaskManager: This is a feature that's also not > > > supported in the DSM right now, is it? We should state that fact in the > > > FLIP if we mentioned that we don't want to implement it for the FSGM. > > > > > > Best, > > > Matthias > > > > > > [1] > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-156%3A+Runtime+Interfaces+for+Fine-Grained+Resource+Requirements > > > > > > On Tue, Mar 7, 2023 at 8:58 AM Yangze Guo wrote: > > > > > > > Hi Weihua, > > > > > > > > Thanks for driving this. As Xintong mentioned, this was a technical > > > > debt from FLIP-56. > > > > > > > > The latest version of FLIP sounds good, +1 from my side. As a > > > > contributor to this component, I'm willing to assist with the review > > > > process. Feel free to reach me if you need help. > > > > > > > > Best, > > > > Yangze Guo > > > > > > > > On Tue, Mar 7, 2023 at 1:47 PM Weihua Hu > > wrote: > > > > > > > > > > Hi, > > > > > > > > > > @David @Matthias > > > > > There are a few days after hearing your thoughts. I would like to > > know > > > if > > > > > there are any other concerns about this FLIP. > > > > > > > > > > > > > > > Best, > > > > > Weihua > > > > > > > > > > > > > > > On Mon, Mar 6, 2023 at 7:53 PM Weihua Hu > > > wrote: > > > > > > > > > > > > > > > > > Thanks Shammon, > > > > > > > > > > > > I've updated FLIP to add this redundant Task Manager limitation. > > > > > > > > > > > > > > > > > > Best, > > > > > > Weihua > > > > > > > > > > > > > > > > > > On Mon, Mar 6, 2023 at 5:07 PM
Re: [ANNOUNCE] Release 1.17.0, release candidate #1
I guess there is something wrong with signature while checking it i got this gpg: assuming signed data in 'flink-1.17.0-bin-scala_2.12.tgz' gpg: Signature made Wed 08 Mar 2023 03:59:17 PM CET gpg:using RSA key A1BD477F79D036D2C30CA7DBCA8AEEC2F6EB040B gpg: BAD signature from "Qingsheng Ren (CODE SIGNING KEY) " [unknown] there is no such issue with flink-1.17.0-src.tgz.asc On Thu, Mar 9, 2023 at 8:32 PM Martijn Visser wrote: > Hi Yingjie, > > Thanks for the test and identifying the issue, this is super helpful! > > To all others, please continue your testing on this RC so that if there are > more blockers to be found, we can fix them with the next RC and have > (hopefully) a successful vote on it. > > Best regards, > > Martijn > > On Thu, Mar 9, 2023 at 4:54 PM Yingjie Cao > wrote: > > > Hi community and release managers: > > > > When testing the release candidate #1 for batch scenario, I found a > > potential deadlock issue of blocking shuffle. I have created a ticket [1] > > for it and marked it as blocker. I will fix it no later than tomorrow. > > > > [1] https://issues.apache.org/jira/browse/FLINK-31386 > > > > Best regards, > > Yingjie > > > > Qingsheng Ren 于2023年3月9日周四 13:51写道: > > > > > Hi everyone, > > > > > > The RC1 for Apache Flink 1.17.0 has been created. This RC currently is > > for > > > preview only to facilitate the integrated testing since the release > > > announcement is still under review. The voting process will be > triggered > > > once the announcement is ready. It has all the artifacts that we would > > > typically have for a release, except for the release note and the > website > > > pull request for the release announcement. > > > > > > The following contents are available for your review: > > > > > > - The preview source release and binary convenience releases [1], which > > > are signed with the key with fingerprint A1BD477F79D036D2C30C [2]. > > > - all artifacts that would normally be deployed to the Maven > > > Central Repository [3]. > > > - source code tag "release-1.17.0-rc1" [4] > > > > > > Your help testing the release will be greatly appreciated! And we'll > > > create the voting thread as soon as all the efforts are finished. > > > > > > [1] https://dist.apache.org/repos/dist/dev/flink/flink-1.17.0-rc1 > > > [2] https://dist.apache.org/repos/dist/release/flink/KEYS > > > [3] > > https://repository.apache.org/content/repositories/orgapacheflink-1591 > > > [4] https://github.com/apache/flink/releases/tag/release-1.17.0-rc1 > > > > > > Best regards, > > > Qingsheng, Leonard, Matthias and Martijn > > > > > > -- Best regards, Sergey
[jira] [Created] (FLINK-31400) Add autoscaler integration for Iceberg source
Maximilian Michels created FLINK-31400: -- Summary: Add autoscaler integration for Iceberg source Key: FLINK-31400 URL: https://issues.apache.org/jira/browse/FLINK-31400 Project: Flink Issue Type: New Feature Components: Autoscaler, Kubernetes Operator Reporter: Maximilian Michels Fix For: kubernetes-operator-1.5.0 A very critical part in the scaling algorithm is setting the source processing correctly such that the Flink pipeline can keep up with the ingestion rate. The autoscaler does that by looking at the {{pendingRecords}} Flink source metric. Even if that metric is not available, the source can still be sized according to the busyTimeMsPerSecond metric, but there will be no backlog information available. For Kafka, the autoscaler also determines the number of partitions to avoid scaling higher than the maximum number of partitions. In order to support a wider range of use cases, we should investigate an integration with the Iceberg source. As far as I know, it does not expose the pedingRecords metric, nor does the autoscaler know about other constraints, e.g. the maximum number of open files. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (FLINK-31399) AdaptiveScheduler is able to handle changes in job resource requirements
David Morávek created FLINK-31399: - Summary: AdaptiveScheduler is able to handle changes in job resource requirements Key: FLINK-31399 URL: https://issues.apache.org/jira/browse/FLINK-31399 Project: Flink Issue Type: Sub-task Components: Runtime / Coordination Reporter: David Morávek Assignee: David Morávek -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (FLINK-31398) Don't wrap with TemporaryClassLoaderContext in OperationExecutor
luoyuxia created FLINK-31398: Summary: Don't wrap with TemporaryClassLoaderContext in OperationExecutor Key: FLINK-31398 URL: https://issues.apache.org/jira/browse/FLINK-31398 Project: Flink Issue Type: Improvement Components: Connectors / Hive, Table SQL / Client Reporter: luoyuxia -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (FLINK-31397) Introduce write-once hash lookup store
Jingsong Lee created FLINK-31397: Summary: Introduce write-once hash lookup store Key: FLINK-31397 URL: https://issues.apache.org/jira/browse/FLINK-31397 Project: Flink Issue Type: Sub-task Components: Table Store Reporter: Jingsong Lee Assignee: Jingsong Lee Fix For: table-store-0.4.0 Introduce interface for lookup changelog producer: {code:java} /** * A key-value store for lookup, key-value store should be single binary file written once and ready * to be used. This factory provide two interfaces: * * * Writer: written once to prepare binary file. * Reader: lookup value by key bytes. * */ public interface LookupStoreFactory { LookupStoreWriter createWriter(File file) throws IOException; LookupStoreReader createReader(File file) throws IOException; } {code} We can convert remote columnar data to local lookup store, and ready to be used to lookup. -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: [Vote] FLIP-298: Unifying the Implementation of SlotManager
Thanks Weihua. +1 (non-binding) Best, Zhanghao Chen From: Weihua Hu Sent: Thursday, March 9, 2023 13:27 To: dev Subject: [Vote] FLIP-298: Unifying the Implementation of SlotManager Hi Everyone, I would like to start the vote on FLIP-298: Unifying the Implementation of SlotManager [1]. The FLIP was discussed in this thread [2]. This FLIP aims to unify the implementation of SlotManager in order to reduce maintenance costs. The vote will last for at least 72 hours (03/14, 15:00 UTC+8) unless there is an objection or insufficient votes. Thank you all. [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-298%3A+Unifying+the+Implementation+of+SlotManager [2]https://lists.apache.org/thread/ocssfxglpc8z7cto3k8p44mrjxwr67r9 Best, Weihua
Re: [Vote] FLIP-298: Unifying the Implementation of SlotManager
+1 (binding) On Fri, Mar 10, 2023 at 8:17 AM David Morávek wrote: > +1 (binding) > > Best, > D. > > On Fri, Mar 10, 2023 at 4:49 AM Yuxin Tan wrote: > > > Thanks, Weihua! > > +1 (non-binding) > > > > Best, > > Yuxin > > > > > > weijie guo 于2023年3月10日周五 11:29写道: > > > > > +1 (binding) > > > > > > Best regards, > > > > > > Weijie > > > > > > > > > Shammon FY 于2023年3月10日周五 11:02写道: > > > > > > > Thanks weihua, +1 (non-binding) > > > > > > > > Best, > > > > Shammon > > > > > > > > On Fri, Mar 10, 2023 at 10:32 AM Xintong Song > > > > > wrote: > > > > > > > > > +1 (binding) > > > > > > > > > > Best, > > > > > > > > > > Xintong > > > > > > > > > > > > > > > > > > > > On Thu, Mar 9, 2023 at 1:28 PM Weihua Hu > > > wrote: > > > > > > > > > > > Hi Everyone, > > > > > > > > > > > > I would like to start the vote on FLIP-298: Unifying the > > > Implementation > > > > > > of SlotManager [1]. The FLIP was discussed in this thread [2]. > > > > > > > > > > > > This FLIP aims to unify the implementation of SlotManager in > > > > > > order to reduce maintenance costs. > > > > > > > > > > > > The vote will last for at least 72 hours (03/14, 15:00 UTC+8) > > > > > > unless there is an objection or insufficient votes. Thank you > all. > > > > > > > > > > > > [1] > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-298%3A+Unifying+the+Implementation+of+SlotManager > > > > > > [2] > > https://lists.apache.org/thread/ocssfxglpc8z7cto3k8p44mrjxwr67r9 > > > > > > > > > > > > Best, > > > > > > Weihua > > > > > > > > > > > > > > > > > > > > >
Re: [DISCUSS] FLIP-300: Add targetColumns to DynamicTableSink#Context to solve the null overwrite problem of partial-insert
Hi everyone, If there are no other questions or concerns for the FLIP[1], I'd like to start the vote next Monday(3.13). [1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=240885081 Best, Lincoln Lee Jing Ge 于2023年3月10日周五 05:59写道: > Hi Lincoln, > > sounds good to me. Thanks! > > Best, > Jing > > On Wed, Mar 8, 2023 at 2:01 PM Lincoln Lee wrote: > > > Hi Jing, > > Agree with you that using formal terms can be easier to users, I've > updated > > the FLIP[1], since this is only one of the application scenarios for > > partial insert, our java doc for the corresponding interface will > describe > > the partial insert message itself from a generic point of view, WDTY? > > > > @Jacky thanks for your feedback! > > here are my thoughts for the two questions: > > for this scenario, I don't think the planner should report an error. We > > cannot assume that such usage will necessarily result in errors or that > > users are unaware of potential risks (just like in a database, similar > > operations are not prompted with errors). In the streaming scenario, > > regarding the risks associated with the multi-insert operation with > > overlapping fields, we may consider expanding the plan advice (FLIP-280 > has > > just added possibilities to support this) to prompt users instead of > > rejecting the operation with an error. > > > 1. if the two insert into with same columns, the result is not > > nondeterminism. will it check in planner and throw exception > > > > yes, not all connectors support partial insert. Therefore, the > introduction > > of this interface is only intended as additional information for the > > connectors that need it. The new `targetColumns` only provide the column > > list information corresponding to the statement according to the SQL > > standard, and existing connectors do not need to make any passive changes > > by default. > > > 2. some sink connectors can not supports it like queue such as kafka > > compacted topic. will also it check in planner and throw exception > > > > welcome your feedback! > > > > > > [1] > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=240885081 > > > > > > Best, > > Lincoln Lee > > > > > > Jacky Lau 于2023年3月8日周三 20:11写道: > > > > > Thanks for bringing this up. this is a good feature. but i have two > > > questions: > > > 1. if the two insert into with same columns, the result is > > > not nondeterminism. will it check in planner and throw exception > > > 2. some sink connectors can not supports it like queue such as kafka > > > compacted topic. will also it check in planner and throw exception > > > > > > Lincoln Lee 于2023年3月7日周二 14:53写道: > > > > > > > Hi Aitozi, > > > > > > > > Thanks for your feedback! Yes, including HBase and JDBC connector, > > they > > > > can be considered for support in the next step (JDBC as as a standard > > > > protocol supported not only in traditional databases, but also in > more > > > and > > > > more new types of storage). Considering the ongoing externalizing of > > > > connectors and the release cycles of the connectors are decoupled > with > > > the > > > > release cycle of Flink, we can initiate corresponding support issues > > for > > > > specific connectors to follow up on support after finalizing the API > > > > changes, WDYT? > > > > > > > > Best, > > > > Lincoln Lee > > > > > > > > > > > > Hang Ruan 于2023年3月7日周二 12:14写道: > > > > > > > > > Hi, Lincoln, > > > > > > > > > > Thanks for bringing this up. It looks good to me. I also agree with > > > > > Jingsong's suggestion. > > > > > > > > > > Best, > > > > > Hang > > > > > > > > > > Jingsong Li 于2023年3月7日周二 11:15写道: > > > > > > > > > > > Wow, we have 300 FLIPs... > > > > > > > > > > > > Thanks Lincoln, > > > > > > > > > > > > Have you considered returning an Optional? > > > > > > > > > > > > Empty array looks a little weird to me. > > > > > > > > > > > > Best, > > > > > > Jingsong > > > > > > > > > > > > On Tue, Mar 7, 2023 at 10:32 AM Aitozi > > wrote: > > > > > > > > > > > > > > Hi Lincoln, > > > > > > > Thank you for sharing this FLIP. Overall, it looks good to > > me. > > > I > > > > > have > > > > > > > one question: with the introduction of this interface, > > > > > > > will any existing Flink connectors need to be updated in order > to > > > > take > > > > > > > advantage of its capabilities? For example, HBase. > > > > > > > > > > > > > > yuxia 于2023年3月7日周二 10:01写道: > > > > > > > > > > > > > > > Thanks. It makes sense to me. > > > > > > > > > > > > > > > > Best regards, > > > > > > > > Yuxia > > > > > > > > > > > > > > > > - 原始邮件 - > > > > > > > > 发件人: "Lincoln Lee" > > > > > > > > 收件人: "dev" > > > > > > > > 发送时间: 星期一, 2023年 3 月 06日 下午 10:26:26 > > > > > > > > 主题: Re: [DISCUSS] FLIP-300: Add targetColumns to > > > > > > DynamicTableSink#Context > > > > > > > > to solve the null overwrite problem of partial-insert > > > > > > > > > > > > > > > > hi yuxia, > > > > > > > > > > > > > > > > Thanks for your
Re: [DISCUSS] FLIP-301: Hybrid Shuffle supports Remote Storage
Hi Yuxin, Thanks for creating this FLIP! The idea of tiered storage looks good. Instead of choosing one from multiple storages, it can help to balance between performance, cost and stability. It also has the potential to adaptively select proper tiers according to more runtime information, to achieve better performance and ease of use. I have a question about the tier finding of data reading. In the FLIP it proposes that the Read Client asks each storage tier whether a given segment exists in it, from higher priority tiers to lower priority ones. I'm a bit concerned about the cost of it, especially when data are written to low priority tiers. Do you have any evaluation of it? Is it possible to let the Reader Client know the location of the next segment when it has finished reading one segment? Or maybe just let it know whether the next segment is located in the same tier, if we can have the assumption that tier changing would not be very frequent. Thanks, Zhu Weihua Hu 于2023年3月10日周五 11:52写道: > > Thanks Yuxin for your explanation. > > That sounds reasonable. Looking forward to the new shuffle. > > > Best, > Weihua > > > On Fri, Mar 10, 2023 at 11:48 AM Yuxin Tan wrote: > > > Hi, Weihua, > > Thanks for the questions and the ideas. > > > > > 1. How many performance regressions would there be if we only > > used remote storage? > > > > The new architecture can support to use remote storage only, but this > > FLIP target is to improve job stability. And the change in the FLIP has > > been significantly complex and the goal of the first version is to update > > Hybrid Shuffle to the new architecture and support remote storage as > > a supplement. The performance of this version is not the first priority, > > so we haven’t tested the performance of using only remote storage. > > If there are indeed regressions, we will keep optimizing the performance > > of the remote storages and improve it until only remote storage is > > available in the production environment. > > > > > 2. Shall we move the local data to remote storage if the producer is > > finished for a long time? > > > > I agree that it is a good idea, which can release task manager resources > > more timely. But moving data from TM local disk to remote storage needs > > more detailed discussion and design, and it is easier to implement it based > > on the new architecture. Considering the complexity, the target focus, and > > the iteration cycle of the FLIP, we decide that the details are not > > included > > in the first version. We will extend and implement them in the subsequent > > versions. > > > > Best, > > Yuxin > > > > > > Weihua Hu 于2023年3月9日周四 11:22写道: > > > > > Hi, Yuxin > > > > > > Thanks for driving this FLIP. > > > > > > The remote storage shuffle could improve the stability of Batch jobs. > > > > > > In our internal scenario, we use a hybrid cluster to run both > > > Streaming(high priority) > > > and Batch jobs(low priority). When there is not enough resources(such as > > > cpu usage > > > reaches a threshold), the batch containers will be evicted. So this will > > > cause some re-run > > > of batch tasks. > > > > > > It would be a great help if the remote storage could address this. So I > > > have a few questions. > > > > > > 1. How many performance regressions would there be if we only used remote > > > storage? > > > > > > 2. In current design, the shuffle data segment will write to one kind of > > > storage tier. > > > Shall we move the local data to remote storage if the producer is > > finished > > > for a long time? > > > So we can release the idle task manager with no shuffle data on it. This > > > may help to reduce > > > the resource usage when producer parallelism is larger than consume. > > > > > > Best, > > > Weihua > > > > > > > > > On Thu, Mar 9, 2023 at 10:38 AM Yuxin Tan > > wrote: > > > > > > > Hi, Junrui, > > > > Thanks for the suggestions and ideas. > > > > > > > > > If they are fixed, I suggest that FLIP could provide clearer > > > > explanations. > > > > I have updated the FLIP and described the segment size more clearly. > > > > > > > > > can we provide configuration options for users to manually adjust the > > > > sizes? > > > > The segment size can be configured if necessary. But considering that > > if > > > we > > > > exposed these parameters prematurely, it may be difficult to modify the > > > > implementation later because the user has already used the configs. We > > > > can make these internal configs or fixed values when implementing the > > > first > > > > version, I think we can use either of these two ways, because they are > > > > internal and do not affect the public APIs. > > > > > > > > Best, > > > > Yuxin > > > > > > > > > > > > Junrui Lee 于2023年3月8日周三 00:24写道: > > > > > > > > > Hi Yuxin, > > > > > > > > > > This FLIP looks quite reasonable. Flink can solve the problem of > > Batch > > > > > shuffle by > > > > > combining local and remote storage, and can use fixed local disks for > > > >
[jira] [Created] (FLINK-31396) Occasional inaccurate timeout time calculation with System.nanotime in batch read buffer pool
Yuxin Tan created FLINK-31396: - Summary: Occasional inaccurate timeout time calculation with System.nanotime in batch read buffer pool Key: FLINK-31396 URL: https://issues.apache.org/jira/browse/FLINK-31396 Project: Flink Issue Type: Bug Components: Runtime / Network Reporter: Yuxin Tan When running TPC-DS tests, I encountered the read buffer request timeout because of configuring too less read buffers. But I found the timeout time may be less than 5m occasionally, 5m is the expected time. I read the docs of System.nanotime, the docs say that t1 < t0 should not be used, because of the possibility of numerical overflow. I tested the System.currentTimeMillis and it can work as expected. -- This message was sent by Atlassian Jira (v8.20.10#820010)