You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/07 22:50:43 UTC

[GitHub] [flink] charles-tan opened a new pull request, #20205: [FLINK-28250][Connector/Kafka] Fix exactly-once Kafka Sink out of memory

charles-tan opened a new pull request, #20205:
URL: https://github.com/apache/flink/pull/20205

   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   The goal of this PR is to address a memory leak where the Kafka Producer does not get cleaned up after kafka commit is successful.
   
   
   ## Brief change log
   
   [7304f9d](https://github.com/charles-tan/flink/commit/7304f9dd3c2c65dd985f5de831496b091b58b556): recycles producer after successful kafka commit by adding a finally block to the existing try/catch.
   
   
   ## Verifying this change
   
   Manually verified the change by running a 1 node cluster with 1 JobManagers and 1 TaskManagers, running the following job for several hours without any memory issues:
   ```
   public class FlinkTest {
       public void main(String[] args) throws Exception {
           StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
           KafkaSource<String> source = KafkaSource.<String>builder()
               .setBootstrapServers("localhost:9092")
               .setTopics("input")
               .setGroupId("my-group" + System.currentTimeMillis())
               .setStartingOffsets(OffsetsInitializer.earliest())
               .setValueOnlyDeserializer(new SimpleStringSchema())
               .build();
           DataStream<String> sourceStream = env.fromSource(
               source,
               WatermarkStrategy.forMonotonousTimestamps(), "Kafka Source");
           KafkaRecordSerializationSchema<String> serializer = KafkaRecordSerializationSchema.builder()
               .setValueSerializationSchema(new SimpleStringSchema())
               .setTopic("output")
               .build();
           Properties sinkProps = new Properties();
           sinkProps.put("[transaction.timeout.ms](http://transaction.timeout.ms/)", 180000);
           KafkaSink<String> sink = KafkaSink.<String>builder()
               .setBootstrapServers("localhost:9092")
               .setKafkaProducerConfig(sinkProps)
               .setRecordSerializer(serializer)
               .setDeliverGuarantee(DeliveryGuarantee.EXACTLY_ONCE)
               .build();
           sourceStream.sinkTo(sink);
           env.enableCheckpointing(10000);
           env.getCheckpointConfig().setCheckpointTimeout(60000);
           env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
           env.getCheckpointConfig().setMinPauseBetweenCheckpoints(500L);
           env.getCheckpointConfig().setTolerableCheckpointFailureNumber(1);
           env.execute("tester");
       }
   }
   ```
   Took heap dumps using `jmap` tool and noticed the taskmanager memory was stable. Link to the relevant mailing list thread: https://lists.apache.org/thread/c86cd8qyqb6qxy639hkzbozkwv2qxk84.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] charles-tan commented on pull request #20205: [FLINK-28250][Connector/Kafka] Fix exactly-once Kafka Sink out of memory

Posted by GitBox <gi...@apache.org>.
charles-tan commented on PR #20205:
URL: https://github.com/apache/flink/pull/20205#issuecomment-1183564908

   > Thanks for the patch @charles-tan ! LGTM. Could you squash all commits together? I'll merge the PR then.
   > 
   > Also we need to backport the patch to 1.15. Could you help to create the backport PR?
   
   @PatrickRen thanks for the review! I've squashed the commits and created a backport patch for 1.15 [here](https://github.com/apache/flink/pull/20266).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #20205: [FLINK-28250][Connector/Kafka] Fix exactly-once Kafka Sink out of memory

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20205:
URL: https://github.com/apache/flink/pull/20205#issuecomment-1178341140

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7304f9dd3c2c65dd985f5de831496b091b58b556",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7304f9dd3c2c65dd985f5de831496b091b58b556",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7304f9dd3c2c65dd985f5de831496b091b58b556 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] PatrickRen commented on a diff in pull request #20205: [FLINK-28250][Connector/Kafka] Fix exactly-once Kafka Sink out of memory

Posted by GitBox <gi...@apache.org>.
PatrickRen commented on code in PR #20205:
URL: https://github.com/apache/flink/pull/20205#discussion_r916412313


##########
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java:
##########
@@ -112,6 +112,8 @@ public void commit(Collection<CommitRequest<KafkaCommittable>> requests)
                         e);
                 recyclable.ifPresent(Recyclable::close);

Review Comment:
   What about removing this line as the `recyclable` will finally be closed anyway? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] PatrickRen merged pull request #20205: [FLINK-28250][Connector/Kafka] Fix exactly-once Kafka Sink out of memory

Posted by GitBox <gi...@apache.org>.
PatrickRen merged PR #20205:
URL: https://github.com/apache/flink/pull/20205


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] PatrickRen commented on a diff in pull request #20205: [FLINK-28250][Connector/Kafka] Fix exactly-once Kafka Sink out of memory

Posted by GitBox <gi...@apache.org>.
PatrickRen commented on code in PR #20205:
URL: https://github.com/apache/flink/pull/20205#discussion_r917610468


##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java:
##########
@@ -85,6 +87,26 @@ public void testFailJobOnUnknownFatalError() throws IOException, InterruptedExce
         }
     }
 
+    @Test
+    public void testKafkaCommitterClosesProducer() throws IOException, InterruptedException {
+        Properties properties = getProperties();
+        FlinkKafkaInternalProducer<Object, Object> producer =
+                mock(FlinkKafkaInternalProducer.class);

Review Comment:
   Unfortunately we are not allowed to use Mockito according to the [code style guide](https://flink.apache.org/contributing/code-style-and-quality-common.html#avoid-mockito---use-reusable-test-implementations). 
   
   The best alternative would be having a fully functioned `MockFlinkKafkaInternalProducer` similar to `MockProducer` provided by Kafka, but I'm OK just having an anonymous class extending the `FlinkKafkaInternalProducer`  and overriding the `commitTransaction` method for now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on pull request #20205: [FLINK-28250][Connector/Kafka] Fix exactly-once Kafka Sink out of memory

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #20205:
URL: https://github.com/apache/flink/pull/20205#issuecomment-1180026255

   @flinkbot run azure


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org