You are viewing a plain text version of this content. The canonical link for it is here.
- Re: [DISCUSS] Introduce partitioning strategies to Table/SQL - posted by Benchao Li <li...@apache.org> on 2020/09/01 02:50:51 UTC, 4 replies.
- [jira] [Created] (FLINK-19105) Table API Sample Code Error - posted by "weizheng (Jira)" <ji...@apache.org> on 2020/09/01 02:56:00 UTC, 0 replies.
- Re: [DISCUSS] FLIP-139: General Python User-Defined Aggregate Function on Table API - posted by Wei Zhong <we...@gmail.com> on 2020/09/01 03:15:36 UTC, 1 replies.
- Re: [DISCUSS] FLIP-136: Improve interoperability between DataStream and Table API - posted by Jark Wu <im...@gmail.com> on 2020/09/01 04:19:22 UTC, 38 replies.
- [jira] [Created] (FLINK-19106) Add more timeout options for remote function specs - posted by "Tzu-Li (Gordon) Tai (Jira)" <ji...@apache.org> on 2020/09/01 05:19:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19107) Add basic checkpoint and recovery config keys to template flink-conf.yaml - posted by "Tzu-Li (Gordon) Tai (Jira)" <ji...@apache.org> on 2020/09/01 06:14:00 UTC, 0 replies.
- [DISCUSS] FLIP-140: Introduce bounded style execution for keyed streams - posted by Dawid Wysakowicz <dw...@apache.org> on 2020/09/01 06:49:06 UTC, 20 replies.
- [jira] [Created] (FLINK-19108) Stop expanding the identifiers with scope aliased by the system with 'EXPR$' prefix - posted by "Danny Chen (Jira)" <ji...@apache.org> on 2020/09/01 07:19:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19109) Split Reader eats chained periodic watermarks - posted by "David Anderson (Jira)" <ji...@apache.org> on 2020/09/01 07:32:00 UTC, 0 replies.
- Re: Next Stateful Functions Release - posted by Igal Shilman <ig...@ververica.com> on 2020/09/01 08:24:10 UTC, 0 replies.
- Flink Speedcenter worker machine replaced - posted by Khachatryan Roman <kh...@gmail.com> on 2020/09/01 08:56:29 UTC, 1 replies.
- Re: FileSystemHaServices and BlobStore - posted by Yang Wang <da...@gmail.com> on 2020/09/01 08:58:00 UTC, 4 replies.
- Re: [DISCUSS] FLIP-134: DataStream Semantics for Bounded Input - posted by Aljoscha Krettek <al...@apache.org> on 2020/09/01 11:46:48 UTC, 9 replies.
- [jira] [Created] (FLINK-19110) Flatten current PyFlink documentation structure - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/01 12:07:00 UTC, 0 replies.
- Re: [DISCUSS] FLIP-141: Intra-Slot Managed Memory Sharing - posted by Aljoscha Krettek <al...@apache.org> on 2020/09/01 12:24:41 UTC, 13 replies.
- [jira] [Created] (FLINK-19111) Flink Docker image creates checkpoints as root user and hits permission denied afterwards - posted by "Tobias Kaymak (Jira)" <ji...@apache.org> on 2020/09/01 13:01:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19112) No access to metric group in ScalarFunction when optimizing - posted by "Nico Kruber (Jira)" <ji...@apache.org> on 2020/09/01 16:27:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19113) Add support for checkpointing with selectable inputs - posted by "Roman Khachatryan (Jira)" <ji...@apache.org> on 2020/09/01 17:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19114) Introduce Expression class in the Python Table API - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/02 01:53:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19115) Null value fields cannot be ignored in ElasticSearch Sink - posted by "McClone (Jira)" <ji...@apache.org> on 2020/09/02 02:03:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19116) Support more kinds of data for expressions.lit in the Python Table API - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/02 02:03:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19117) FLINK-18620 - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/02 02:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19118) Support Expression in the operations of the Python Table API - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/02 02:08:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19119) Update the documentation to use Expression instead of strings in the Python Table API - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/02 02:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19120) Failed to build Flink with HBase2.1 even using the latest Flink release version. - posted by "Tang Yan (Jira)" <ji...@apache.org> on 2020/09/02 03:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19121) Avoid access HDFS in HiveBulkWriterFactory - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/02 05:57:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19122) Prometheus scrape generates huge scrape target. - posted by "Harold Dost III (Jira)" <ji...@apache.org> on 2020/09/02 08:25:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19123) TestStreamEnvironment does not use shared MiniCluster for executeAsync() - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/02 09:53:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19124) Some JobClient methods are not ergonomic, require ClassLoader argument - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/02 11:04:00 UTC, 0 replies.
- [VOTE] FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API) - posted by Aljoscha Krettek <al...@apache.org> on 2020/09/02 11:22:51 UTC, 6 replies.
- [DISCUSS] Releasing Flink 1.11.2 - posted by Zhu Zhu <re...@gmail.com> on 2020/09/02 12:19:12 UTC, 17 replies.
- [jira] [Created] (FLINK-19125) Avoid memory fragmentation when running flink docker image - posted by "Yun Tang (Jira)" <ji...@apache.org> on 2020/09/02 13:24:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19126) Failed to run job in yarn-cluster mode due to No Executor found. - posted by "Tang Yan (Jira)" <ji...@apache.org> on 2020/09/02 14:16:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19127) Provide a replacement of StreamExecutionEnvironment.createRemoteEnvironment for TableEnvironment - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/02 15:24:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19128) Remove the runtime execution configuration in sql-client-defaults.yaml - posted by "Jark Wu (Jira)" <ji...@apache.org> on 2020/09/02 15:38:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19129) Helm charts are missing the latest log4j-console file - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/02 16:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19130) Expose backpressure metrics / logs for function dispatcher operator - posted by "Tzu-Li (Gordon) Tai (Jira)" <ji...@apache.org> on 2020/09/03 04:51:00 UTC, 0 replies.
- Re: [DISCUSS] FLIP-137: Support Pandas UDAF in PyFlink - posted by jincheng sun <su...@gmail.com> on 2020/09/03 05:47:10 UTC, 4 replies.
- [RESULT][VOTE] Remove deprecated DataStream#fold and DataStream#split in 1.12 - posted by Dawid Wysakowicz <dw...@apache.org> on 2020/09/03 06:33:51 UTC, 0 replies.
- Re: HadoopOutputFormat has issues with LocalExecutionEnvironment? - posted by Robert Metzger <rm...@apache.org> on 2020/09/03 06:43:14 UTC, 1 replies.
- [jira] [Created] (FLINK-19131) Add py38 support in PyFlink - posted by "sunjincheng (Jira)" <ji...@apache.org> on 2020/09/03 08:46:00 UTC, 0 replies.
- [VOTE] FLIP-141: Intra-Slot Managed Memory Sharing - posted by Xintong Song <to...@gmail.com> on 2020/09/03 10:42:34 UTC, 12 replies.
- [jira] [Created] (FLINK-19132) Failed to start jobs for consuming Secure Kafka after cluster restart - posted by "Olivier Zembri (Jira)" <ji...@apache.org> on 2020/09/03 11:48:00 UTC, 0 replies.
- Bug: Kafka producer always writes to partition 0, because KafkaSerializationSchemaWrapper does not call open() method of FlinkKafkaPartitioner - posted by "DONG, Weike" <ky...@connect.hku.hk> on 2020/09/03 12:15:08 UTC, 2 replies.
- [jira] [Created] (FLINK-19133) User provided partitioners are not initialized correctly - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/03 12:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19134) Fix the converter of array coder for Python DataStream API. - posted by "Shuiqiang Chen (Jira)" <ji...@apache.org> on 2020/09/03 12:49:00 UTC, 0 replies.
- Re: [DISCUSS] FLIP-138: Declarative Resource management - posted by Till Rohrmann <tr...@apache.org> on 2020/09/03 16:30:45 UTC, 2 replies.
- [VOTE] FLIP-138: Declarative Resource Management - posted by Till Rohrmann <tr...@apache.org> on 2020/09/03 16:37:23 UTC, 3 replies.
- [jira] [Created] (FLINK-19135) (Stream)ExecutionEnvironment.execute() should not throw ExecutionException - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/03 16:44:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19136) MetricsAvailabilityITCase.testReporter failed with "Could not satisfy the predicate within the allowed time" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/04 01:54:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19137) Bump Apache Parquet to 1.11.1 - posted by "ABC (Jira)" <ji...@apache.org> on 2020/09/04 02:26:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19138) Python UDF supports directly specifying input_types as DataTypes.ROW - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/04 02:58:00 UTC, 0 replies.
- [VOTE] FLIP-137: Support Pandas UDAF in PyFlink - posted by Xingbo Huang <hx...@gmail.com> on 2020/09/04 03:12:48 UTC, 7 replies.
- Merge SupportsComputedColumnPushDown and SupportsWatermarkPushDown - posted by Shengkai Fang <fs...@gmail.com> on 2020/09/04 06:58:14 UTC, 7 replies.
- [jira] [Created] (FLINK-19139) flink-conf.yml read values from environment variables - posted by "Mohammad Hossein Gerami (Jira)" <ji...@apache.org> on 2020/09/04 08:00:00 UTC, 0 replies.
- Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records - posted by Timo Walther <tw...@apache.org> on 2020/09/04 08:48:02 UTC, 41 replies.
- [jira] [Created] (FLINK-19140) Join with Table Function (UDTF) SQL example is incorrect - posted by "Benchao Li (Jira)" <ji...@apache.org> on 2020/09/04 09:26:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19141) Flink Job Submitted on Yarn Does not Work - posted by "Yunhui (Jira)" <ji...@apache.org> on 2020/09/04 11:14:00 UTC, 0 replies.
- Re: [DISCUSS] FLIP-36 - Support Interactive Programming in Flink Table API - posted by Timo Walther <tw...@apache.org> on 2020/09/04 11:29:43 UTC, 8 replies.
- [jira] [Created] (FLINK-19142) Investigate slot hijacking from preceding pipelined regions after failover - posted by "Andrey Zagrebin (Jira)" <ji...@apache.org> on 2020/09/04 12:12:00 UTC, 0 replies.
- Apply for contributor permissions - posted by jun su <su...@gmail.com> on 2020/09/04 12:28:50 UTC, 2 replies.
- Re: [DISCUSS] FLIP-33: Standardize connector metrics - posted by Becket Qin <be...@gmail.com> on 2020/09/04 14:56:10 UTC, 9 replies.
- [DISCUSS] FLINK-19089: improvement of ClosableBlockingQueue - posted by dugenkui <du...@foxmail.com> on 2020/09/06 01:38:21 UTC, 0 replies.
- [jira] [Created] (FLINK-19143) YARNSessionCapacitySchedulerITCase#testVCoresAreSetCorrectlyAndJobManagerHostnameAreShownInWebInterfaceAndDynamicPropertiesAndYarnApplicationNameAndTaskManagerSlots failed to submit JobGraph: "Connection refused: localhost/127.0.0.1:8081" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/06 13:42:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19144) Error when writing to partitioned table with s3 FileSystem - posted by "Pei He (Jira)" <ji...@apache.org> on 2020/09/06 20:33:00 UTC, 0 replies.
- 退订dev - posted by 高丽丽 <li...@koubei.com> on 2020/09/07 02:58:11 UTC, 1 replies.
- [jira] [Created] (FLINK-19145) Add PyFlink-walkthrough in Flink playground. - posted by "Shuiqiang Chen (Jira)" <ji...@apache.org> on 2020/09/07 04:09:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19146) createMiniBatchTrigger() use OR ,table.exec.mini-batch.size and table.exec.mini-batch.allow-latency - posted by "badqiu (Jira)" <ji...@apache.org> on 2020/09/07 06:20:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19147) Support AutoClose in CliClient - posted by "darion yaphet (Jira)" <ji...@apache.org> on 2020/09/07 06:35:00 UTC, 0 replies.
- [VOTE] FLIP-139: General Python User-Defined Aggregate Function on Table API - posted by Wei Zhong <we...@gmail.com> on 2020/09/07 06:37:56 UTC, 5 replies.
- Flink stateful functions : compensating callback to invoked functions on a timeout - posted by Mazen Ezzeddine <ma...@etu.unice.fr> on 2020/09/07 06:43:49 UTC, 0 replies.
- [jira] [Created] (FLINK-19148) Table crashed in Flink Table API & SQL Docs - posted by "Leonard Xu (Jira)" <ji...@apache.org> on 2020/09/07 06:50:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19149) Compacted Kafka Topic can be interpreted as Changelog Stream - posted by "Konstantin Knauf (Jira)" <ji...@apache.org> on 2020/09/07 07:21:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19150) Behaviour change after migration from 1.9 to 1.11 - posted by "Jiayi Liao (Jira)" <ji...@apache.org> on 2020/09/07 07:49:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19151) Flink does not normalize container resource with correct configurations when Yarn FairScheduler is used - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/07 08:09:00 UTC, 0 replies.
- Re: [DISCUSS][Release 1.12] Stale blockers and build instabilities - posted by Robert Metzger <rm...@apache.org> on 2020/09/07 08:37:32 UTC, 2 replies.
- [jira] [Created] (FLINK-19152) Remove Kafka 0.10.x and 0.11.x connectors - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/07 09:00:12 UTC, 0 replies.
- [RESULT][VOTE] FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API) - posted by Aljoscha Krettek <al...@apache.org> on 2020/09/07 09:26:58 UTC, 0 replies.
- [jira] [Created] (FLINK-19153) FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API) - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/07 09:32:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19155) ResultPartitionTest is unstable - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/07 09:37:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19154) Always clean up HA data when application completion - posted by "Husky Zeng (Jira)" <ji...@apache.org> on 2020/09/07 09:37:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19156) Migration of transactionIdHint in Kafka is never applied - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/07 10:22:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19157) Jdbc PhoenixDialect - posted by "Zhou Zach (Jira)" <ji...@apache.org> on 2020/09/07 10:34:00 UTC, 0 replies.
- h - posted by 56...@qq.com on 2020/09/07 12:11:23 UTC, 0 replies.
- Re: [jira] [Created] (FLINK-16595) Support extra hadoop filesystem URLs for which to request delegation tokens - posted by Husky Zeng <56...@qq.com> on 2020/09/07 12:12:01 UTC, 0 replies.
- [jira] [Created] (FLINK-19158) Revisit java e2e download timeouts - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/07 12:14:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19159) Using Scalafmt to format scala source code - posted by "darion yaphet (Jira)" <ji...@apache.org> on 2020/09/07 13:08:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19160) When backpressured AsyncWaitOperator/ContinousFileReaderOperator are not idle - posted by "Konstantin Knauf (Jira)" <ji...@apache.org> on 2020/09/07 14:00:14 UTC, 0 replies.
- Re: Flink stateful functions : compensating callback to invoked functions on a timeout - posted by Dawid Wysakowicz <dw...@apache.org> on 2020/09/07 16:17:54 UTC, 2 replies.
- [jira] [Created] (FLINK-19161) Port File Sources to FLIP-27 API - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/07 18:00:03 UTC, 0 replies.
- [jira] [Created] (FLINK-19162) Allow Split Reader based sources to reuse record batches - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/07 18:04:00 UTC, 0 replies.
- [ANNOUNCE] Weekly Community Update 2020/36 - posted by Konstantin Knauf <kn...@apache.org> on 2020/09/07 19:44:43 UTC, 0 replies.
- [jira] [Created] (FLINK-19163) Add building py38 wheel package of PyFlink in Azure CI - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/08 08:45:00 UTC, 0 replies.
- 1.11.1 Hive connector doesn't work with Hive 1.0 or 1.1 - posted by Rui Li <li...@gmail.com> on 2020/09/08 11:19:47 UTC, 3 replies.
- [jira] [Created] (FLINK-19164) Release scripts break other dependency versions unintentionally - posted by "Serhat Soydan (Jira)" <ji...@apache.org> on 2020/09/08 13:35:00 UTC, 0 replies.
- [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing - posted by Seth Wiesman <sj...@gmail.com> on 2020/09/08 17:05:23 UTC, 39 replies.
- [jira] [Created] (FLINK-19165) Clean up the UnilateralSortMerger - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/08 17:24:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19166) StreamingFileWriter should register Listener before the initialization of buckets - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/09 03:06:00 UTC, 0 replies.
- [RESULT][VOTE] FLIP-137: Support Pandas UDAF in PyFlink - posted by Xingbo Huang <hx...@gmail.com> on 2020/09/09 03:26:13 UTC, 0 replies.
- [jira] [Created] (FLINK-19168) Upgrade Kafka client version - posted by "darion yaphet (Jira)" <ji...@apache.org> on 2020/09/09 06:51:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19167) Proccess Function Example could not work - posted by "tinny cat (Jira)" <ji...@apache.org> on 2020/09/09 06:51:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19169) Support Pandas UDAF in PyFlink (FLIP-137) - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/09 07:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19170) Parameter naming error - posted by "sulei (Jira)" <ji...@apache.org> on 2020/09/09 07:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19171) K8s Resource Manager may lead to resource leak after pod deleted - posted by "Yi Tang (Jira)" <ji...@apache.org> on 2020/09/09 07:30:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19172) [AbstractFileStateBackend] - posted by "Alessio Savi (Jira)" <ji...@apache.org> on 2020/09/09 08:25:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19173) Add Pandas Batch Group Aggregation Function Operator - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/09 10:01:00 UTC, 0 replies.
- [CANCEL][VOTE] FLIP-134: DataStream Semantics for Bounded Input - posted by Aljoscha Krettek <al...@apache.org> on 2020/09/09 11:40:06 UTC, 0 replies.
- [DISCUSS] Deprecate and remove UnionList OperatorState - posted by Aljoscha Krettek <al...@apache.org> on 2020/09/09 14:09:19 UTC, 5 replies.
- [jira] [Created] (FLINK-19174) idleTimeMsPerSecond can report incorrect values if task is blocked for more then 60 seconds - posted by "Piotr Nowojski (Jira)" <ji...@apache.org> on 2020/09/09 14:46:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19175) Tests in JoinITCase do not test BroadcastHashJoin - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/09 16:34:00 UTC, 0 replies.
- [DISCUSS] Support source/sink parallelism config in Flink sql - posted by admin <17...@163.com> on 2020/09/09 17:18:56 UTC, 4 replies.
- [jira] [Created] (FLINK-19176) Support ScalaPB as a message payload serializer in Stateful Functions - posted by "Galen Warren (Jira)" <ji...@apache.org> on 2020/09/09 22:13:01 UTC, 0 replies.
- add support for ScalaPB-based message-payload serialization to Stateful Functions? - posted by Galen Warren <ga...@cvillewarrens.com> on 2020/09/09 22:26:46 UTC, 1 replies.
- [RESULT][VOTE] FLIP-141: Intra-Slot Managed Memory Sharing - posted by Xintong Song <to...@gmail.com> on 2020/09/10 01:58:48 UTC, 0 replies.
- [jira] [Created] (FLINK-19177) FLIP-141: Intra-Slot Managed Memory Sharing - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/10 02:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19178) Introduce the memory weights configuration option - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/10 02:28:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19180) Make RocksDB respect the calculated fraction - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/10 02:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19179) Implement the new fraction calculation logic - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/10 02:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19182) Update document for intra-slot managed memory sharing - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/10 02:30:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19181) Make python processes respect the calculated fraction - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/10 02:30:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19183) flink-connector-hive module compile failed with "cannot find symbol: variable TableEnvUtil" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/10 05:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19184) Add Batch Physical Pandas Group Aggregate Rule and RelNode - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/10 06:54:00 UTC, 0 replies.
- [VOTE] Release 1.11.2, release candidate #1 - posted by Zhu Zhu <zh...@apache.org> on 2020/09/10 07:03:58 UTC, 10 replies.
- [RESULT][VOTE] FLIP-139: General Python User-Defined Aggregate Function on Table API - posted by Wei Zhong <we...@gmail.com> on 2020/09/10 07:41:54 UTC, 0 replies.
- [jira] [Created] (FLINK-19185) Support General Python User-Defined Aggregate Function Support on Table API (FLIP-139) - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/10 07:48:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19186) Add Python building blocks to make sure the basic functionality of Pandas Batch Group Aggregation could work - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/10 08:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19187) Add a new basic Table API example - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/10 08:30:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19188) Add a new streaming SQL examples - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/10 08:36:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19189) Enable pipelined scheduling by default - posted by "Zhu Zhu (Jira)" <ji...@apache.org> on 2020/09/10 08:55:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19190) Use camelCase for metric names - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/10 09:26:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19191) Reduce the default number for async operations - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/10 10:31:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19192) Set higher limit on the HTTP connection pool - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/10 11:11:00 UTC, 0 replies.
- [VOTE] FLIP-107: Handling of metadata in SQL connectors - posted by Timo Walther <tw...@apache.org> on 2020/09/10 11:57:44 UTC, 7 replies.
- [jira] [Created] (FLINK-19193) Upgrade migration guidelines to use stop-with-savepoint - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/10 12:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19194) The UDF split and split_index get wrong result - posted by "fa zheng (Jira)" <ji...@apache.org> on 2020/09/10 12:18:00 UTC, 0 replies.
- [VOTE] FLIP-140: Introduce bounded style execution for keyed streams - posted by Dawid Wysakowicz <dw...@apache.org> on 2020/09/10 13:04:01 UTC, 4 replies.
- [DISCUSS] FLIP-143: Unified Sink API - posted by Guowei Ma <gu...@gmail.com> on 2020/09/10 13:43:58 UTC, 60 replies.
- [ANNOUNCE] Feature freeze for Stateful Functions 2.2.0 - posted by "Tzu-Li (Gordon) Tai" <tz...@apache.org> on 2020/09/10 14:38:11 UTC, 0 replies.
- [DISCUSS] Drop Scala 2.11 - posted by Seth Wiesman <sj...@gmail.com> on 2020/09/10 14:51:53 UTC, 9 replies.
- [jira] [Created] (FLINK-19195) question on security vulnerabilities in flink - posted by "Miguel Costa (Jira)" <ji...@apache.org> on 2020/09/10 16:14:00 UTC, 0 replies.
- Notification of analysis on publicly available project data - posted by Griselda Cuevas <gr...@apache.org> on 2020/09/10 16:48:22 UTC, 0 replies.
- [jira] [Created] (FLINK-19196) FlinkSQL aggregation function to aggregate array typed column to Map/Multiset - posted by "sam lin (Jira)" <ji...@apache.org> on 2020/09/10 19:02:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19197) Add documentation for PersistedStateRegistry - posted by "Tzu-Li (Gordon) Tai (Jira)" <ji...@apache.org> on 2020/09/11 03:40:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19198) Invalid default value for consume-start-offset in Hive streaming - posted by "Paul Lin (Jira)" <ji...@apache.org> on 2020/09/11 03:54:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19199) Add execution attempt ID to feedback channel keys - posted by "Tzu-Li (Gordon) Tai (Jira)" <ji...@apache.org> on 2020/09/11 04:01:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19200) UNIX_TIMESTAMP function support return in millisecond - posted by "leslieyuan (Jira)" <ji...@apache.org> on 2020/09/11 06:15:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19201) PyFlink e2e tests is instable and failed with "Connection broken: OSError" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/11 07:59:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19202) HadoopS3FileSystemITCase.testSimpleFileWriteAndRead failed with "FileNotFoundException: No such file or directory: s3://***/temp/tests-7ebc11df-9160-4403-85a7-c484da44f335/test.txt" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/11 08:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19203) Use Flink-*-scala2.12 variants for StateFun - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/11 11:14:00 UTC, 0 replies.
- Re: How to schedule Flink Batch Job periodically or daily - posted by Robert Metzger <rm...@apache.org> on 2020/09/11 11:20:58 UTC, 0 replies.
- [jira] [Created] (FLINK-19204) Improve warning during a Table.execute() - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/11 12:09:00 UTC, 0 replies.
- [VOTE] FLIP-136: Improve interoperability between DataStream and Table API - posted by Timo Walther <tw...@apache.org> on 2020/09/11 14:19:41 UTC, 6 replies.
- [jira] [Created] (FLINK-19205) SourceReaderContext should give access to Configuration and Hostbame - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/11 16:31:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19206) Add an ability to set ownerReference manually in Kubernetes - posted by "Mike Kaplinskiy (Jira)" <ji...@apache.org> on 2020/09/12 06:09:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19207) TtlListState#IteratorWithCleanup support remove method - posted by "dalongliu (Jira)" <ji...@apache.org> on 2020/09/12 07:27:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19208) Refine the tuple's equals - posted by "darion yaphet (Jira)" <ji...@apache.org> on 2020/09/12 11:57:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19209) Single bucket - posted by "Michał Strużek (Jira)" <ji...@apache.org> on 2020/09/12 22:26:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19210) TPC-DS end-to-end test (Blink planner) failed with "NoResourceAvailableException: Could not allocate the required slot within slot request timeout" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/13 00:27:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19211) SQL Read Decimal In Parquet Error - posted by "nyq (Jira)" <ji...@apache.org> on 2020/09/13 01:17:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19212) Failed to serialize the result for RPC call : requestTaskManagerInfo. - posted by "Ken (Jira)" <ji...@apache.org> on 2020/09/13 04:42:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19213) Update the Chinese documentation - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/14 08:02:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19214) Update the flink-web - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/14 08:04:00 UTC, 0 replies.
- [ANNOUNCE] New Apache Flink Committer - Niels Basjes - posted by Robert Metzger <rm...@apache.org> on 2020/09/14 08:37:01 UTC, 28 replies.
- [VOTE] FLIP-134: Batch execution for the DataStream API - posted by Aljoscha Krettek <al...@apache.org> on 2020/09/14 12:25:09 UTC, 6 replies.
- [jira] [Created] (FLINK-19215) "Resuming Savepoint (rocks, scale down, rocks timers) end-to-end test" failed with "Dispatcher REST endpoint has not started within a timeout of 20 sec" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/14 12:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19216) Reduce the duplicate argument check - posted by "darion yaphet (Jira)" <ji...@apache.org> on 2020/09/14 13:23:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19217) Functions repeated extend Serializable interface - posted by "darion yaphet (Jira)" <ji...@apache.org> on 2020/09/14 14:06:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19218) Remove inconsistent host logic for LocalFileSystem - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/14 14:21:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19219) Run JobManager initialization in a separate thread, to make it cancellable - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/14 14:29:00 UTC, 0 replies.
- Java and Scala code format - posted by Darion Yaphet <da...@gmail.com> on 2020/09/14 14:50:39 UTC, 2 replies.
- [VOTE] FLIP-142: Disentangle StateBackends from Checkpointing - posted by Seth Wiesman <sj...@gmail.com> on 2020/09/14 15:05:18 UTC, 9 replies.
- [jira] [Created] (FLINK-19220) Add a way to close internal resources - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/14 15:09:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19221) Exploit LocatableFileStatus from Hadoop - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/14 15:40:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19222) Elevate external SDKs - posted by "Seth Wiesman (Jira)" <ji...@apache.org> on 2020/09/14 16:23:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19223) Simplify Availability Future Model in Base Connector - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/14 21:14:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19224) Provide an easy way to read window state - posted by "Seth Wiesman (Jira)" <ji...@apache.org> on 2020/09/14 21:34:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19225) Improve code and logging in SourceReaderBase - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/14 21:56:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19226) [Kinesis] [EFO] Connector reaches default max attempts for describeStream and describeStreamConsumer when parallelism is high - posted by "Hong Liang Teoh (Jira)" <ji...@apache.org> on 2020/09/14 23:39:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19227) The catalog is still created after opening failed in catalog registering - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/15 01:44:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19228) Avoid accessing FileSystem in client for file system connector - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/15 02:01:01 UTC, 0 replies.
- [jira] [Created] (FLINK-19229) Support ValueState and Python UDAF on blink stream planner - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/15 02:03:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19232) Support MapState and MapView for Python UDAF - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/15 02:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19230) Support Python UDAF on blink batch planner - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/15 02:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19231) Support ListState and ListView for Python UDAF - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/15 02:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19234) Support FILTER KeyWord for Python UDAF - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/15 02:12:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19235) Support mixed use with built-in aggs for Python UDAF - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/15 02:12:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19233) Support DISTINCT KeyWord for Python UDAF - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/15 02:12:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19236) Optimize the performance of Python UDAF - posted by "Wei Zhong (Jira)" <ji...@apache.org> on 2020/09/15 02:13:00 UTC, 0 replies.
- [ANNOUNCE] New Apache Flink Committer - Arvid Heise - posted by Zhijiang <wa...@aliyun.com.INVALID> on 2020/09/15 02:38:41 UTC, 27 replies.
- [jira] [Created] (FLINK-19237) LeaderChangeClusterComponentsTest.testReelectionOfJobMaster failed with "NoResourceAvailableException: Could not allocate the required slot within slot request timeout" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/15 03:20:00 UTC, 0 replies.
- [DISCUSS] FLIP-144: Native Kubernetes HA for Flink - posted by Yang Wang <da...@gmail.com> on 2020/09/15 03:25:36 UTC, 10 replies.
- [RESULT][VOTE] Release 1.11.2, release candidate #1 - posted by Zhu Zhu <zh...@apache.org> on 2020/09/15 05:08:24 UTC, 0 replies.
- [jira] [Created] (FLINK-19238) RocksDB performance issue with low managed memory and high parallelism - posted by "Juha Mynttinen (Jira)" <ji...@apache.org> on 2020/09/15 07:04:00 UTC, 0 replies.
- [ANNOUNCE] Weekly Community Update 2020/37 - posted by Konstantin Knauf <kn...@apache.org> on 2020/09/15 07:42:32 UTC, 0 replies.
- [ANNOUNCE] New Apache Flink Committer - Igal Shilman - posted by "Tzu-Li (Gordon) Tai" <tz...@apache.org> on 2020/09/15 08:14:04 UTC, 23 replies.
- [jira] [Created] (FLINK-19239) FlinkKafkaProducer - posted by "zhangyunyun (Jira)" <ji...@apache.org> on 2020/09/15 09:02:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19240) Hello, I have a job with multiple Kafka sources. They all contain certain historical data. If you use the events-time window, it will cause sources with less data to cover more sources through water mark. Is there a solution? - posted by "Ryan (Jira)" <ji...@apache.org> on 2020/09/15 09:26:00 UTC, 0 replies.
- [ANNOUNCE] New Apache Flink Committer - Yun Tang - posted by Yu Li <ca...@gmail.com> on 2020/09/15 10:19:02 UTC, 24 replies.
- [jira] [Created] (FLINK-19241) Forward ClusterEntrypoint ioExecutor to ResourceManager - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/15 11:23:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19242) org.apache.flink.table.api.ValidationException: Cannot resolve field - posted by "henvealf (Jira)" <ji...@apache.org> on 2020/09/15 12:00:06 UTC, 0 replies.
- [jira] [Created] (FLINK-19243) Bump Snakeyaml to 1.27 - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/15 12:12:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19244) CsvRowDataDeserializationSchema throws cast exception : Row length mismatch. - posted by "Ying Z (Jira)" <ji...@apache.org> on 2020/09/15 13:22:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19245) Set default queue capacity for FLIP-27 source handover queue to 2 - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/15 14:06:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19246) TableSourceITCase.testStreamScanParallelism fails on private Azure accounts - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/15 14:18:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19247) Update Chinese documentation after removal of Kafka 0.10 and 0.11 - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/15 14:46:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19248) The main method caused an error: No result found for job, was execute() called before getting the result - posted by "Shiyu Jin (Jira)" <ji...@apache.org> on 2020/09/15 14:52:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19249) Job would wait sometime(~10 min) before failover if some connection broken - posted by "Congxian Qiu(klion26) (Jira)" <ji...@apache.org> on 2020/09/15 14:58:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19250) SplitFetcherManager does not propagate errors correctly - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/15 20:28:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19251) Avoid confusing queue handling in "SplitReader.handleSplitsChanges()" - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/15 21:44:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19252) Jaas file created under io.tmp.dirs - folder not created if not exists - posted by "Koala Lam (Jira)" <ji...@apache.org> on 2020/09/16 00:19:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19253) SourceReaderTestBase.testAddSplitToExistingFetcher hangs - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/16 02:54:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19254) Invalid UTF-8 start byte exception - posted by "Jun Zhang (Jira)" <ji...@apache.org> on 2020/09/16 03:45:00 UTC, 0 replies.
- [VOTE] FLIP-36 - Support Interactive Programming in Flink Table API - posted by Xuannan Su <su...@gmail.com> on 2020/09/16 04:00:00 UTC, 4 replies.
- [ANNOUNCE] New Apache Flink Committer - Godfrey He - posted by Jark Wu <im...@gmail.com> on 2020/09/16 04:19:40 UTC, 23 replies.
- [jira] [Created] (FLINK-19255) Add configuration to make AsyncWaitOperation Chainable - posted by "Kyle (Jira)" <ji...@apache.org> on 2020/09/16 05:06:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19256) Validate Stateful Functions configuration only where necessary - posted by "Tzu-Li (Gordon) Tai (Jira)" <ji...@apache.org> on 2020/09/16 06:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19257) HBase ConnectorDescriptor can not work in Table API of release-1.11 - posted by "liufangliang (Jira)" <ji...@apache.org> on 2020/09/16 06:34:00 UTC, 0 replies.
- KeyedCoProcessFunction, processElement1, processElement2, onTimer timeout - posted by Mazen Ezzeddine <ma...@etu.unice.fr> on 2020/09/16 06:50:10 UTC, 2 replies.
- [jira] [Created] (FLINK-19258) Fix the wrong example of "csv.line-delimiter" in CSV documentation - posted by "Jark Wu (Jira)" <ji...@apache.org> on 2020/09/16 07:27:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19259) Use classloader release hooks with Kinesis producer to avoid metaspace leak - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/16 07:28:00 UTC, 0 replies.
- [DISCUSS] Forwarding Kafka's metrics groups - posted by Dawid Wysakowicz <dw...@apache.org> on 2020/09/16 07:58:55 UTC, 1 replies.
- Connecting two streams and order of their processing - posted by Mazen Ezzeddine <ma...@etu.unice.fr> on 2020/09/16 08:06:21 UTC, 0 replies.
- Re: KeyedCoProcessFunction, processElement1, processElement2, onTimer timeout - posted by David Anderson <da...@alpinegizmo.com> on 2020/09/16 08:13:27 UTC, 1 replies.
- [jira] [Created] (FLINK-19260) Update documentation based on bin/flink output - posted by "Guowei Ma (Jira)" <ji...@apache.org> on 2020/09/16 08:28:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19261) Update document according to RestOptions - posted by "Guowei Ma (Jira)" <ji...@apache.org> on 2020/09/16 08:37:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19262) Can not setParallelism for FLIP-27 source - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/16 08:40:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19263) Enforce alphabetical order in configuration option docs - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/16 11:38:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19264) MiniCluster is flaky with concurrent job execution - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/16 14:23:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19265) Simplify handling of 'NoMoreSplitsEvent' - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/16 16:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19266) Test fail when run azure in AbstractCloseableRegistryTest#testClose - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/16 18:00:11 UTC, 0 replies.
- [jira] [Created] (FLINK-19267) Calculate managed memory fractions for fine grained resource specs - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/17 03:05:00 UTC, 0 replies.
- random fetch - posted by Chenyang Tang <gt...@163.com> on 2020/09/17 03:19:14 UTC, 2 replies.
- [ANNOUNCE] Apache Flink 1.11.2 released - posted by Zhu Zhu <zh...@apache.org> on 2020/09/17 05:29:10 UTC, 7 replies.
- 回复: Re: [ANNOUNCE] New Apache Flink Committer - Godfrey He - posted by "chengyanan1008@foxmail.com" <ch...@foxmail.com> on 2020/09/17 05:34:40 UTC, 0 replies.
- [RESULT][VOTE] FLIP-140: Introduce bounded style execution for keyed streams - posted by Dawid Wysakowicz <dw...@apache.org> on 2020/09/17 06:42:30 UTC, 4 replies.
- [RESULT][VOTE] FLIP-107: Handling of metadata in SQL connectors - posted by Timo Walther <tw...@apache.org> on 2020/09/17 07:06:43 UTC, 0 replies.
- [jira] [Created] (FLINK-19268) Introduce bounded style execution for keyed streams - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/17 07:17:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19269) Make the PushingAsyncDataInput.DataOutput aware of endOfInput - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/17 07:22:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19270) Extract an inteface from Abstract - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/17 07:23:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19271) wrong HOP_PROCTIME output when materialize proctime - posted by "Leonard Xu (Jira)" <ji...@apache.org> on 2020/09/17 07:38:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19272) Add interfaces for FLIP-107 - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/17 08:38:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19273) Update parser module for FLIP-107 - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/17 08:39:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19274) Support FLIP-107 interfaces in planner - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/17 09:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19275) Allow to read and write metadata in Kafka table source/sink - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/17 09:12:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19276) Allow to read metadata for Debezium format - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/17 09:14:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19277) Introduce BatchArrowPythonGroupWindowAggregateFunctionOperator - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/17 09:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19278) Bump Scala Macros Version to 2.1.1 - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/17 09:59:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19279) Remove StatefulFunctionUnvierse cache - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/17 11:13:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19280) The option "sink.buffer-flush.max-rows" for JDBC can't be disabled by set to zero - posted by "Jark Wu (Jira)" <ji...@apache.org> on 2020/09/17 12:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19281) LIKE cannot recognize full table path - posted by "Benchao Li (Jira)" <ji...@apache.org> on 2020/09/17 12:36:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19282) Support watermark push down with WatermarkStrategy - posted by "Shengkai Fang (Jira)" <ji...@apache.org> on 2020/09/17 13:38:00 UTC, 0 replies.
- Avoiding "too many open files" during leftOuterJoin with Flink 1.11/batch - posted by Ken Krugler <kk...@transpac.com> on 2020/09/17 21:06:41 UTC, 2 replies.
- Avoiding "too many open files" during leftOuterJoin with Flink 1.11/batch - now with stack trace - posted by Ken Krugler <kk...@transpac.com> on 2020/09/17 21:15:46 UTC, 0 replies.
- [jira] [Created] (FLINK-19283) Allow subclasses to override/extend FlinkKafkaConsumerBase checkpoint methods - posted by "Adrian Kreuziger (Jira)" <ji...@apache.org> on 2020/09/17 23:12:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19284) Add documentation about how to use Python UDF in the Java Table API - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/18 01:28:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19285) Kafka sql connector fixed partitioner not work - posted by "limbo (Jira)" <ji...@apache.org> on 2020/09/18 05:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19286) Improve pipelined region scheduling performance - posted by "Zhu Zhu (Jira)" <ji...@apache.org> on 2020/09/18 05:48:00 UTC, 0 replies.
- [DISCUSS] What is the correct site version? - posted by Jingsong Li <ji...@gmail.com> on 2020/09/18 05:51:11 UTC, 5 replies.
- [DISCUSS] Release flink-shaded 12.0 - posted by Chesnay Schepler <ch...@apache.org> on 2020/09/18 06:25:37 UTC, 5 replies.
- [jira] [Created] (FLINK-19287) Fix minor version in flink docs - posted by "Zhu Zhu (Jira)" <ji...@apache.org> on 2020/09/18 06:46:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19288) Make InternalTimeServiceManager an interface - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/18 08:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19289) K8s resource manager terminated pod garbage collection - posted by "Yi Tang (Jira)" <ji...@apache.org> on 2020/09/18 08:18:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19290) Add documentation for Stateful Function's Flink DataStream SDK - posted by "Tzu-Li (Gordon) Tai (Jira)" <ji...@apache.org> on 2020/09/18 08:35:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19291) Meeting `SchemaParseException` when I use `AvroSchemaConverter` converts flink logical type - posted by "xiaozilong (Jira)" <ji...@apache.org> on 2020/09/18 08:57:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19292) HiveCatalog should support specifying Hadoop conf dir with configuration - posted by "Rui Li (Jira)" <ji...@apache.org> on 2020/09/18 12:49:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19293) RocksDB last_checkpoint.state_size grows endlessly until savepoint/restore - posted by "Thomas Wozniakowski (Jira)" <ji...@apache.org> on 2020/09/18 13:04:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19294) Support key and value formats in Kafka connector - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/18 14:21:00 UTC, 0 replies.
- Timed out patterns handling using MATCH_RECOGNIZE - posted by Kosma Grochowski <ko...@getindata.com> on 2020/09/18 14:48:23 UTC, 1 replies.
- Can you unify the language ? - posted by 490548661 <49...@qq.com> on 2020/09/19 14:21:03 UTC, 0 replies.
- [jira] [Created] (FLINK-19295) YARNSessionFIFOITCase.checkForProhibitedLogContents found a log with prohibited string - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/20 13:37:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19296) RetryingCallback is not aware of task cancaltion - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/20 20:07:00 UTC, 0 replies.
- Re: Timed out patterns handling using MATCH_RECOGNIZE - posted by Jark Wu <im...@gmail.com> on 2020/09/21 03:12:02 UTC, 2 replies.
- [VOTE] FLIP-33: Standardize connector metrics - posted by Becket Qin <be...@gmail.com> on 2020/09/21 03:31:18 UTC, 5 replies.
- [jira] [Created] (FLINK-19297) Make ResultPartitionWriter record-oriented - posted by "Yingjie Cao (Jira)" <ji...@apache.org> on 2020/09/21 03:32:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19298) Maven enforce goal dependency-convergence failed on flink-json - posted by "Jark Wu (Jira)" <ji...@apache.org> on 2020/09/21 03:41:00 UTC, 0 replies.
- [RESULT][VOTE] FLIP-36 - Support Interactive Programming in Flink Table API - posted by Xuannan Su <su...@gmail.com> on 2020/09/21 04:02:22 UTC, 0 replies.
- [jira] [Created] (FLINK-19299) NettyShuffleEnvironmentBuilder#setBufferSize does not take effect - posted by "Yingjie Cao (Jira)" <ji...@apache.org> on 2020/09/21 04:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19300) Timer loss after restoring from savepoint - posted by "Xiang Gao (Jira)" <ji...@apache.org> on 2020/09/21 04:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19301) Improve the package structure of Python DataStream API - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/21 06:25:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19302) Flushing of BoundedBlockingResultPartition should finish current BufferBuilder - posted by "Yingjie Cao (Jira)" <ji...@apache.org> on 2020/09/21 06:43:00 UTC, 0 replies.
- Re: Can you unify the language ? - posted by Timo Walther <tw...@apache.org> on 2020/09/21 06:54:21 UTC, 0 replies.
- [VOTE] Apache Flink Stateful Functions 2.2.0, release candidate #1 - posted by "Tzu-Li (Gordon) Tai" <tz...@apache.org> on 2020/09/21 06:56:26 UTC, 2 replies.
- [jira] [Created] (FLINK-19303) Disable WAL in RocksDB recovery - posted by "Juha Mynttinen (Jira)" <ji...@apache.org> on 2020/09/21 07:04:00 UTC, 0 replies.
- [RESULT][VOTE] FLIP-134: Batch execution for the DataStream API - posted by Aljoscha Krettek <al...@apache.org> on 2020/09/21 07:20:35 UTC, 0 replies.
- [RESULT][VOTE] FLIP-138: Declarative Resource Management - posted by Chesnay Schepler <ch...@apache.org> on 2020/09/21 07:39:49 UTC, 0 replies.
- [jira] [Created] (FLINK-19304) Add feature toggle - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 07:56:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19305) Setup Azure build for running core and flink-tests - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 07:57:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19306) Implement DeclarativeSlotManager - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:00:09 UTC, 0 replies.
- [jira] [Created] (FLINK-19307) Add RequirementsTracker - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:06:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19308) Add SlotTracker - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19309) Add TaskExecutorAllocator - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19310) Add new TaskManagerSlot implementation - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:12:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19311) Add ResourceRequirement(s) - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:14:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19312) Introduce BufferWritingResultPartition which wraps the logic of writing buffers to ResultSubpartition - posted by "Yingjie Cao (Jira)" <ji...@apache.org> on 2020/09/21 08:19:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19313) Implement declarative SlotPool - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:42:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19314) Add compatibility layer - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:43:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19315) Move slot-bookkeeping into separate component - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/21 08:55:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19316) FLIP-134: Batch execution for the DataStream API - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/21 09:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19317) Make EventTime the default StreamTimeCharacteristic - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/21 09:08:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19318) Deprecate timeWindow() operations in DataStream API - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/21 09:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19319) Deprecate StreamExecutionEnvironment.setStreamTimeCharacteristic() - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/21 09:18:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19320) Remove clearBuffers from the public interfaces of RecordWriter - posted by "Yingjie Cao (Jira)" <ji...@apache.org> on 2020/09/21 09:18:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19321) CollectSinkFunction does not define serialVersionUID - posted by "Fabian Hueske (Jira)" <ji...@apache.org> on 2020/09/21 09:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19322) Specified parallelism in create table operation - posted by "YufeiLiu (Jira)" <ji...@apache.org> on 2020/09/21 09:37:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19323) Small optimization of network layer record serialization - posted by "Yingjie Cao (Jira)" <ji...@apache.org> on 2020/09/21 09:53:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19324) Map requested/allocated containers with priority on YARN - posted by "Xintong Song (Jira)" <ji...@apache.org> on 2020/09/21 10:00:04 UTC, 0 replies.
- [jira] [Created] (FLINK-19325) Optimize the consumed time for checkpoint completion - posted by "Congxian Qiu(klion26) (Jira)" <ji...@apache.org> on 2020/09/21 11:02:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19326) Allow explicitly configuring time behaviour on CEP PatternStream - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/21 11:04:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19327) Helm charts specify a too low value for the job manager's heap size - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/21 12:49:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19328) flink sql - posted by "cellen (Jira)" <ji...@apache.org> on 2020/09/21 13:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19329) FunctionGroupOperator#dispose might throw an NPE - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/21 14:13:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19330) Recovery with async operations fails due to unitialized runtimeContext - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/21 18:58:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19331) State processor api has native resouce leak when working with RocksDB - posted by "Seth Wiesman (Jira)" <ji...@apache.org> on 2020/09/21 21:17:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19332) Special characters issue using Kinesis Data Analytics for Apache Flink - posted by "Zekun Yu (Jira)" <ji...@apache.org> on 2020/09/22 00:47:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19333) Introduce BatchArrowPythonOverWindowAggregateFunctionOperator - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/22 01:51:00 UTC, 0 replies.
- Automatically get udf's resource files from hdfs when running a job that uses hive-udf - posted by Husky Zeng <56...@qq.com> on 2020/09/22 01:57:59 UTC, 1 replies.
- [jira] [Created] (FLINK-19334) Translate page 'Streaming File Sink' into Chinese - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/22 02:24:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19335) Automatically get udf's resource files from hdfs when running a job that uses hive-udf - posted by "Husky Zeng (Jira)" <ji...@apache.org> on 2020/09/22 02:44:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19336) EncodingUtils#encodeObjectToString should propagate inner exception - posted by "Benchao Li (Jira)" <ji...@apache.org> on 2020/09/22 03:04:01 UTC, 0 replies.
- [jira] [Created] (FLINK-19337) Make a little bit improvement for PyFlink package structure and class name. - posted by "sunjincheng (Jira)" <ji...@apache.org> on 2020/09/22 04:00:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19338) New source interface cannot unregister unregistered source - posted by "Arvid Heise (Jira)" <ji...@apache.org> on 2020/09/22 05:18:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19339) Support Avro's unions with logical types - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/22 06:54:00 UTC, 0 replies.
- Kafka connection issues - posted by Ramya Ramamurthy <ha...@gmail.com> on 2020/09/22 06:57:51 UTC, 4 replies.
- [ANNOUNCE] Weekly Community Update 2020/38 - posted by Konstantin Knauf <kn...@apache.org> on 2020/09/22 07:41:57 UTC, 0 replies.
- Does Flink support such a feature currently? - posted by Roc Marshal <fl...@126.com> on 2020/09/22 07:43:11 UTC, 1 replies.
- [jira] [Created] (FLINK-19340) AggregateITCase.testListAggWithDistinct failed with "expected: but was:" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/22 08:15:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19341) Update API module for FLIP-107 - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/22 08:37:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19342) stop overriding convertFrom() in FlinkPlannerImpl after upgrade calcite to 1.23 - posted by "Leonard Xu (Jira)" <ji...@apache.org> on 2020/09/22 08:39:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19343) FLIP-36: Support Interactive Programming in Flink - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 09:25:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19344) DispatcherResourceCleanupTest#testJobSubmissionUnderSameJobId is unstable on Azure Pipeline - posted by "Yingjie Cao (Jira)" <ji...@apache.org> on 2020/09/22 09:28:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19345) Introduce File streaming sink compaction - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/22 09:50:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19346) Generate and put ClusterPartitionDescriptor of ClusterPartition in JobResult when job finishes - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:06:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19347) Generate InputGateDeploymentDescriptor from a JobVertex with ClusterPartitionDescriptor - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:06:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19349) StreamGraph handle CacheSource and CacheSink - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19348) Introduce CacheSource and CacheSink - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19350) StreamingJobGraphGenerator generate job graph with cached node - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:08:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19351) StreamingJobGraphGenerator set the caching node to BoundedBlockingType - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:08:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19352) Add cache() method to Table - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19355) Add close() method to TableEnvironment - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19354) Add invalidateCache() method in CachedTable - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19353) BlinkPlanner translate and optimize CacheOperation - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/22 10:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19356) Introduce FileLifeCycleListener to Buckets - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/22 10:28:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19357) Introduce createBucketWriter to BucketsBuilder - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/22 10:32:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19358) when submit job on application mode with HA,the jobid will be 0000000000 - posted by "Jun Zhang (Jira)" <ji...@apache.org> on 2020/09/22 11:08:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19359) Restore from Checkpoint fails if checkpoint folders is corrupt/partial - posted by "Arpith Prakash (Jira)" <ji...@apache.org> on 2020/09/22 11:17:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19360) Flink fails if JAVA_HOME contains spaces - posted by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/09/22 11:42:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19361) Create a synchronized metastore client to talk to a remote HMS - posted by "Rui Li (Jira)" <ji...@apache.org> on 2020/09/22 13:15:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19362) Remove confusing comment for `DOT` operator codegen - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/22 14:03:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19363) Code of split method grows beyond 64 KB - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/22 16:50:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19364) Add Batch Physical Pandas Group Window Aggregate Rule and RelNode - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/23 02:04:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19365) Migrate Hive table source to new source interface - posted by "Rui Li (Jira)" <ji...@apache.org> on 2020/09/23 03:11:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19366) [umbrella] Migrate Filesystem and Hive to new Table connector interface - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/23 03:16:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19367) Migrate Filesystem source to FLIP-27 source interface - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/23 03:18:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19368) TableEnvHiveConnectorITCase fails with Hive-3.x - posted by "Rui Li (Jira)" <ji...@apache.org> on 2020/09/23 03:42:00 UTC, 0 replies.
- How to run k8s-related end-to-end tests locally? - posted by Di Xu <sc...@gmail.com> on 2020/09/23 05:29:23 UTC, 2 replies.
- [jira] [Created] (FLINK-19369) BlobClientTest.testGetFailsDuringStreamingForJobPermanentBlob hangs - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/23 06:26:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19370) FileSourceTextLinesITCase.testContinuousTextFileSource failed - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/23 06:44:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19371) Support table transformations in SQL - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/23 07:54:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19372) Support Pandas Batch Over Window Aggregation - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/23 08:03:00 UTC, 0 replies.
- [CANCEL][VOTE] FLIP-136: Improve interoperability between DataStream and Table API - posted by Timo Walther <tw...@apache.org> on 2020/09/23 08:13:27 UTC, 0 replies.
- [VOTE] Apache Flink Stateful Functions 2.2.0, release candidate #2 - posted by "Tzu-Li (Gordon) Tai" <tz...@apache.org> on 2020/09/23 08:29:08 UTC, 6 replies.
- [jira] [Created] (FLINK-19373) `jmx.server.port`'s document is missing. - posted by "Guowei Ma (Jira)" <ji...@apache.org> on 2020/09/23 08:58:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19374) update the `table.exec.state.ttl`'s documentation - posted by "Guowei Ma (Jira)" <ji...@apache.org> on 2020/09/23 09:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19375) The `kubernetes.secrets` and `kubernetes.env.secretKeyRef`'s document is missing - posted by "Guowei Ma (Jira)" <ji...@apache.org> on 2020/09/23 09:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19376) `table.generated-code.max-length` and `table.sql-dialec` 's document is needed to be updated - posted by "Guowei Ma (Jira)" <ji...@apache.org> on 2020/09/23 09:15:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19377) Task can swallow test exceptions which hides test failures - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/23 09:34:00 UTC, 0 replies.
- [DISCUSS] FLIP-146: Improve new TableSource and TableSink interfaces - posted by Jingsong Li <ji...@gmail.com> on 2020/09/23 10:22:51 UTC, 12 replies.
- [jira] [Created] (FLINK-19378) Running "./bin/flink run" without HADOOP_CLASSPATH to submit job to running YARN session fails - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/23 12:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19379) Submitting job to running YARN session fails - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/23 12:09:00 UTC, 0 replies.
- [DISCUSS] Support registering custom JobStatusListeners when scheduling a job - posted by 季文昊 <pr...@gmail.com> on 2020/09/23 12:31:09 UTC, 2 replies.
- [jira] [Created] (FLINK-19380) Add support for a gRPC transport for the RequestReply protocol. - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/23 13:17:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19381) Fix docs about relocatable savepoints - posted by "Nico Kruber (Jira)" <ji...@apache.org> on 2020/09/23 13:19:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19382) Introducing ReplayableSourceStateBackend - posted by "Theo Diefenthal (Jira)" <ji...@apache.org> on 2020/09/23 15:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19383) Per Partition State - posted by "Theo Diefenthal (Jira)" <ji...@apache.org> on 2020/09/23 15:54:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19384) Source API exception signatures are inconsistent - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/23 15:56:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19385) Channel recovery may deadlock - posted by "Roman Khachatryan (Jira)" <ji...@apache.org> on 2020/09/23 18:28:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19386) Remove legacy table planner dependecy from connectors and formats - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/23 23:52:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19387) ConnectedComponentsWithDeferredUpdateITCase.testJobWithoutObjectReuse deadlock - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/24 01:51:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19388) Streaming bucketing end-to-end test failed with "Number of running task managers has not reached 4 within a timeout of 40 sec" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/24 03:57:00 UTC, 0 replies.
- [DISCUSS] Move Hive document to "Table & SQL Connectors" from "Table API & SQL" - posted by Jingsong Li <ji...@gmail.com> on 2020/09/24 04:15:48 UTC, 6 replies.
- [jira] [Created] (FLINK-19389) Error determining python major version to start fileserver during e-2-e tests - posted by "Di Xu (Jira)" <ji...@apache.org> on 2020/09/24 05:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19390) mysql jdbc join error java.lang.ClassCastException: java.math.BigInteger cannot be cast to java.lang.Long - posted by "badqiu (Jira)" <ji...@apache.org> on 2020/09/24 07:15:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19391) Deadlock during partition update - posted by "Arvid Heise (Jira)" <ji...@apache.org> on 2020/09/24 07:24:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19392) Detect the execution mode based on the sources in the job. - posted by "Kostas Kloudas (Jira)" <ji...@apache.org> on 2020/09/24 07:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19393) Translate the 'SQL Hints' page of 'Table API & SQL' into Chinese - posted by "Roc Marshal (Jira)" <ji...@apache.org> on 2020/09/24 08:43:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19394) Translate the 'Monitoring Checkpointing' page of 'Debugging & Monitoring' into Chinese - posted by "Roc Marshal (Jira)" <ji...@apache.org> on 2020/09/24 08:49:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19395) Replace SqlConversionException with either TableException or ValidationException - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/24 09:40:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19396) Fix properties type cast error - posted by "zhangmeng (Jira)" <ji...@apache.org> on 2020/09/24 10:51:00 UTC, 0 replies.
- [VOTE] FLIP-143: Unified Sink API - posted by Guowei Ma <gu...@gmail.com> on 2020/09/24 12:13:15 UTC, 7 replies.
- [jira] [Created] (FLINK-19397) Offering error handling for the SerializationSchema analogous to DeserializationSchema - posted by "Matthias (Jira)" <ji...@apache.org> on 2020/09/24 12:54:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19398) Hive connector fails with IllegalAccessError if submitted as usercode - posted by "Fabian Hueske (Jira)" <ji...@apache.org> on 2020/09/24 13:03:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19399) Add Python AsyncRequestReplyHandler docs - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/24 14:53:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19400) Removed unused BufferPoolOwner - posted by "Arvid Heise (Jira)" <ji...@apache.org> on 2020/09/24 15:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19401) Job stuck in restart loop due to "Could not find registered job manager" - posted by "Steven Zhen Wu (Jira)" <ji...@apache.org> on 2020/09/24 16:22:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19402) Metrics for measuring Flink application deployment latency in Yarn - posted by "Yu Yang (Jira)" <ji...@apache.org> on 2020/09/24 21:38:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19403) Support Pandas Stream Group Window Aggregation - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/25 01:47:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19404) Support Pandas Stream Over Window Aggregation - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/25 01:49:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19405) Translate "DataSet Connectors" page of "Connectors" into Chinese - posted by "weizheng (Jira)" <ji...@apache.org> on 2020/09/25 02:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19406) Casting row time to timestamp loses nullability info - posted by "Rui Li (Jira)" <ji...@apache.org> on 2020/09/25 03:17:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19407) Translate "Elasticsearch Connector" page of "Connectors" into Chinese - posted by "魏旭斌 (Jira)" <ji...@apache.org> on 2020/09/25 04:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19408) Update flink-statefun-docker release scripts for cross release Java 8 and 11 - posted by "Tzu-Li (Gordon) Tai (Jira)" <ji...@apache.org> on 2020/09/25 06:28:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19409) The comment for getValue has wrong code in class ListView - posted by "Liu (Jira)" <ji...@apache.org> on 2020/09/25 07:06:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19410) RestAPIStabilityTest does not assert on enum changes - posted by "Matthias (Jira)" <ji...@apache.org> on 2020/09/25 07:18:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19411) MultipleInputStreamTask fails with RuntimeException when its input contains union - posted by "Caizhi Weng (Jira)" <ji...@apache.org> on 2020/09/25 07:41:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19412) Re-layer Python Operation Make it Possible to Provide only Python implementation - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/25 07:49:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19413) Translate "FileSystem" page of "Table & SQL Connectors" into Chinese - posted by "weizheng (Jira)" <ji...@apache.org> on 2020/09/25 08:00:29 UTC, 0 replies.
- [DISCUSS][Code-Style] The approach to implement singleton pattern - posted by Yangze Guo <ka...@gmail.com> on 2020/09/25 08:22:01 UTC, 7 replies.
- [jira] [Created] (FLINK-19414) Introduce ParquetColumnarRowInputFormat - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/25 09:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19415) Move Hive document to "Table & SQL Connectors" from "Table API & SQL" - posted by "Jingsong Lee (Jira)" <ji...@apache.org> on 2020/09/25 09:21:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19416) Support Python datetime object in from_collection of Python DataStream - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/25 12:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19417) Fix the bug of the method from_data_stream in table_environement - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/25 12:16:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19418) Inline PRIMARY KEY constraint should be invalid - posted by "Konstantin Knauf (Jira)" <ji...@apache.org> on 2020/09/25 12:47:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19419) "null-string-literal" does not work in HBaseSource decoder - posted by "CaoZhen (Jira)" <ji...@apache.org> on 2020/09/25 13:14:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19420) Translate "Program Packaging" page of "Managing Execution" into Chinese - posted by "Xiao Huang (Jira)" <ji...@apache.org> on 2020/09/25 14:21:00 UTC, 0 replies.
- [DISCUSS] FLIP-145: Support SQL windowing table-valued function - posted by Jark Wu <im...@gmail.com> on 2020/09/25 14:30:20 UTC, 3 replies.
- [jira] [Created] (FLINK-19421) Support Python UDAF in streaming mode - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/26 01:09:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19422) Avro Confluent Schema Registry nightly end-to-end test failed with "Register operation timed out; error code: 50002" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/26 01:45:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19423) primary key position cause JDBC SQL upsert sink ArrayIndexOutOfBoundsException - posted by "limbo (Jira)" <ji...@apache.org> on 2020/09/26 09:30:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19424) Translate page 'Query Configuration' into Chinese - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/26 11:24:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19425) Correct the usage of BulkWriter#flush and BulkWriter#finish - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/26 13:38:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19426) Streaming File Sink end-to-end test is instable - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/27 01:07:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19427) SplitFetcherTest.testNotifiesWhenGoingIdleConcurrent is instable - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/27 01:13:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19428) Translate "Elasticsearch Connector" page of "DataStream Connectors" into Chinese - posted by "weizheng (Jira)" <ji...@apache.org> on 2020/09/27 01:31:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19429) Translate page 'Data Types' into Chinese - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/27 02:23:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19430) Translate page 'datastream_tutorial' into Chinese - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/27 02:32:00 UTC, 0 replies.
- [RESULT] [VOTE] Apache Flink Stateful Functions 2.2.0, release candidate #2 - posted by "Tzu-Li (Gordon) Tai" <tz...@apache.org> on 2020/09/27 03:21:48 UTC, 0 replies.
- [jira] [Created] (FLINK-19431) Translate "Monitoring REST API" page of "Debugging & Monitoring" into Chinese - posted by "weizheng (Jira)" <ji...@apache.org> on 2020/09/27 03:24:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19432) Whether to capture the updates which don't change any monitored columns - posted by "tinny cat (Jira)" <ji...@apache.org> on 2020/09/27 06:26:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19433) An Error example of FROM_UNIXTIME function in document - posted by "Kyle Zhang (Jira)" <ji...@apache.org> on 2020/09/27 09:02:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19434) Add StreamJobGraphGenerator support for source chaining - posted by "Caizhi Weng (Jira)" <ji...@apache.org> on 2020/09/27 09:20:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19435) jdbc JDBCOutputFormat open function invoke Class.forName(drivername) - posted by "xiaodao (Jira)" <ji...@apache.org> on 2020/09/27 12:26:00 UTC, 0 replies.
- .ORC file reading from Apache Flink +SCALA - posted by Ajay Kumar <ma...@gmail.com> on 2020/09/27 15:28:17 UTC, 1 replies.
- [jira] [Created] (FLINK-19436) TPC-DS end-to-end test (Blink planner) failed during shutdown - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/28 01:00:00 UTC, 0 replies.
- Apply for permission of jira contributor - posted by Xie Billy <bi...@gmail.com> on 2020/09/28 01:04:39 UTC, 2 replies.
- [jira] [Created] (FLINK-19437) FileSourceTextLinesITCase.testContinuousTextFileSource failed with "SimpleStreamFormat is not splittable, but found split end (0) different from file length (198)" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/28 01:05:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19438) Queryable State need implement both read-uncommitted and read-committed - posted by "sheep (Jira)" <ji...@apache.org> on 2020/09/28 02:22:00 UTC, 0 replies.
- [ANNOUNCE] Apache Flink Stateful Functions 2.2.0 released - posted by "Tzu-Li (Gordon) Tai" <tz...@apache.org> on 2020/09/28 06:00:29 UTC, 0 replies.
- [jira] [Created] (FLINK-19439) Performance regression on 24.09.2020 - posted by "Piotr Nowojski (Jira)" <ji...@apache.org> on 2020/09/28 12:17:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19440) Performance regression on 15.09.2020 - posted by "Piotr Nowojski (Jira)" <ji...@apache.org> on 2020/09/28 12:21:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19441) Performance regression on 24.09.2020 - posted by "Arvid Heise (Jira)" <ji...@apache.org> on 2020/09/28 12:42:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19442) Unaligned checkpoints 1.1 - posted by "Arvid Heise (Jira)" <ji...@apache.org> on 2020/09/28 14:31:00 UTC, 0 replies.
- How to clean up resources in a UDF? - posted by Boyuan Zhang <bo...@apache.org> on 2020/09/28 19:44:23 UTC, 0 replies.
- Need help in setting up flink 1.10 - posted by Ravi Sankar Reddy Sangana <Ra...@radware.com> on 2020/09/29 01:03:58 UTC, 1 replies.
- CheckpointedFunction initialization during checkpoint - posted by Teng Fei Liao <te...@gmail.com> on 2020/09/29 02:00:46 UTC, 1 replies.
- [jira] [Created] (FLINK-19443) runtime function 'splitIndex' - posted by "mytang0 (Jira)" <ji...@apache.org> on 2020/09/29 03:57:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19444) flink 1.11 sql group by tumble Window aggregate can only be defined over a time attribute column, but TIMESTAMP(3) encountered - posted by "panxiaohu (Jira)" <ji...@apache.org> on 2020/09/29 06:03:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19445) Several tests for HBase connector 1.4 failed with "NoSuchMethodError: com.google.common.base.Preconditions.checkArgument(ZLjava/lang/String;Ljava/lang/Object;)V" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/29 06:25:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19446) canal-json has a situation that -U and +U are equal, when updating the null field to be non-null - posted by "Zhengchao Shi (Jira)" <ji...@apache.org> on 2020/09/29 06:30:01 UTC, 0 replies.
- [jira] [Created] (FLINK-19447) HBaseConnectorITCase.HBaseTestingClusterAutoStarter failed with "Master not initialized after 200000ms" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/29 06:44:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19448) CoordinatedSourceITCase.testEnumeratorReaderCommunication hangs - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/29 07:08:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19449) LEAD/LAG cannot work correctly in streaming mode - posted by "Benchao Li (Jira)" <ji...@apache.org> on 2020/09/29 09:04:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19450) Optimize the Python CI Test - posted by "Huang Xingbo (Jira)" <ji...@apache.org> on 2020/09/29 09:08:00 UTC, 0 replies.
- Need help in creating Flink Streaming s3 Job for multiple path reader one by one - posted by Satyaa Dixit <sa...@gmail.com> on 2020/09/29 09:27:45 UTC, 1 replies.
- [jira] [Created] (FLINK-19451) Add HELM chart distribution to StateFun release process - posted by "Igal Shilman (Jira)" <ji...@apache.org> on 2020/09/29 09:39:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19452) statistics of group by CDC data is always 1 - posted by "Zhengchao Shi (Jira)" <ji...@apache.org> on 2020/09/29 09:44:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19453) Deprecate old source and sink interfaces - posted by "Timo Walther (Jira)" <ji...@apache.org> on 2020/09/29 10:23:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19454) Translate page 'Importing Flink into an IDE' into Chinese - posted by "wulei0302 (Jira)" <ji...@apache.org> on 2020/09/29 10:46:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19455) Module 'flink-sql-connector-hive-2.3.6' build fail by maven-enforcer-plugin - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/29 12:20:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19456) sql client execute insert sql with comment ahead - posted by "ledong Lin (Jira)" <ji...@apache.org> on 2020/09/29 14:49:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19457) Port NumberSequenceSource to FLIP-27 source interface - posted by "Stephan Ewen (Jira)" <ji...@apache.org> on 2020/09/29 16:30:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19458) ZooKeeperLeaderElectionITCase.testJobExecutionOnClusterWithLeaderChange: ZooKeeper unexpectedly modified - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/29 17:10:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19459) flink-dist won't build locally with newer (3.3+) maven versions - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/29 17:52:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19460) AWS Kinesis Producer EXACTLY_ONCE semantic - posted by "Chris Slotterback (Jira)" <ji...@apache.org> on 2020/09/29 17:55:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19461) yarn-sesson.sh -jm -tm arguments have no effect - posted by "Robert Metzger (Jira)" <ji...@apache.org> on 2020/09/29 17:58:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19462) Checkpoint statistics for unfinished task snapshots - posted by "Nico Kruber (Jira)" <ji...@apache.org> on 2020/09/29 18:23:00 UTC, 0 replies.
- [RESULT][VOTE] FLIP-142: Disentangle StateBackends from Checkpointing - posted by Seth Wiesman <sj...@gmail.com> on 2020/09/29 18:35:14 UTC, 0 replies.
- [jira] [Created] (FLINK-19463) Disentangle StateBackends from Checkpointing - posted by "Seth Wiesman (Jira)" <ji...@apache.org> on 2020/09/29 18:42:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19465) Add CheckpointStorage interface - posted by "Seth Wiesman (Jira)" <ji...@apache.org> on 2020/09/29 19:56:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19464) Rename CheckpointStorage interface to CheckpointStorageAccess - posted by "Seth Wiesman (Jira)" <ji...@apache.org> on 2020/09/29 19:56:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19466) Implement JobManagerCheckpointStorage and FileSystemCheckpointStorage - posted by "Seth Wiesman (Jira)" <ji...@apache.org> on 2020/09/29 19:57:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19467) Implement HashMapStateBackend and EmbeddedRocksDBStateBackend - posted by "Seth Wiesman (Jira)" <ji...@apache.org> on 2020/09/29 19:58:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19468) Metrics not returned when data stream / operator name contains "+" - posted by "Boyang Jerry Peng (Jira)" <ji...@apache.org> on 2020/09/29 22:51:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19469) HBase connector 2.2 failed to download dependencies "org.glassfish:javax.el:jar:3.0.1-b06-SNAPSHOT" - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/30 01:33:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19470) ParquetColumnarRowSplitReader::reachEnd returns false after it has reached end - posted by "Rui Li (Jira)" <ji...@apache.org> on 2020/09/30 03:02:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19471) CVE-2020-7712 is reported for flink-streaming-java_2.11:jar:1.11.1 - posted by "Jeff Hu (Jira)" <ji...@apache.org> on 2020/09/30 05:50:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19472) Implement a one input sorting DataInput - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/30 06:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19473) Implement multi inputs sorting DataInput - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/30 06:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19475) Implement a timer service that holds a single key at a time - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/30 06:30:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19474) Implement a state backends that holds a single key at a time - posted by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/09/30 06:30:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19476) Introduce CacheManager in CatalogManager to keep track of the ClusterPartitionDescriptor - posted by "Xuannan Su (Jira)" <ji...@apache.org> on 2020/09/30 07:32:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19477) Translate page 'python_table_api_connectors' into Chinese - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/30 07:47:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19478) Translate page 'intro_to_table_api' into Chinese - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/30 07:52:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19479) Allow explicitly configuring time behaviour on KeyedStream.intervalJoin() - posted by "Aljoscha Krettek (Jira)" <ji...@apache.org> on 2020/09/30 08:29:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19480) Support RichFunction in Python DataStream API - posted by "Dian Fu (Jira)" <ji...@apache.org> on 2020/09/30 10:31:00 UTC, 0 replies.
- [VOTE] Release flink-shaded 12.0 - posted by Robert Metzger <rm...@apache.org> on 2020/09/30 14:55:49 UTC, 0 replies.
- [jira] [Created] (FLINK-19481) Add support for a flink native GCS FileSystem - posted by "Ben Augarten (Jira)" <ji...@apache.org> on 2020/09/30 18:02:00 UTC, 0 replies.
- [jira] [Created] (FLINK-19482) OrcRowInputFormat does not define serialVersionUID - posted by "hailong wang (Jira)" <ji...@apache.org> on 2020/09/30 18:36:00 UTC, 0 replies.
- [DISCUSS] Introduce MongoDB connector for Flink - posted by Dominik Wosiński <wo...@gmail.com> on 2020/09/30 21:51:42 UTC, 0 replies.