[
https://issues.apache.org/jira/browse/FLINK-8547?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16352283#comment-16352283
]
ASF GitHub Bot commented on FLINK-8547:
---------------------------------------
Github user pnowojski commented on a diff in the pull request:
https://github.com/apache/flink/pull/5400#discussion_r165943880
--- Diff:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BufferBlockerTest.java
---
@@ -0,0 +1,322 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import
org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+import
org.apache.flink.streaming.runtime.io.CreditBasedBufferBlocker.BufferOrEventSequence;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link CreditBasedBufferBlocker}.
+ */
+public class BufferBlockerTest {
--- End diff --
Rename class to `CreditBasedBufferBlockerTest`
> Implement CheckpointBarrierHandler not to spill data for exactly-once based
> on credit-based flow control
> --------------------------------------------------------------------------------------------------------
>
> Key: FLINK-8547
> URL: https://issues.apache.org/jira/browse/FLINK-8547
> Project: Flink
> Issue Type: Sub-task
> Components: Network
> Affects Versions: 1.5.0
> Reporter: zhijiang
> Assignee: zhijiang
> Priority: Major
>
> Currently in exactly-once mode, the {{BarrierBuffer}} would block inputs with
> barriers until all inputs have received the barrier for a given checkpoint.
> To avoid back-pressuring the input streams which may cause distributed
> deadlocks, the {{BarrierBuffer}} has to spill the data in disk files to
> recycle the buffers for blocked channels.
>
> Based on credit-based flow control, every channel has exclusive buffers, so
> it is no need to spill data for avoiding deadlock. Then we implement a new
> {{CheckpointBarrierHandler}} for only buffering the data for blocked channels
> for better performance.
>
> And this new {{CheckpointBarrierHandler}} can also be configured to use or
> not in order to rollback the original mode for unexpected risks.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)