Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
5cebfb76
F
flink
项目概览
doujutun3207
/
flink
与 Fork 源项目一致
从无法访问的项目Fork
通知
24
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
F
flink
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
提交
5cebfb76
编写于
1月 03, 2020
作者:
Z
Zhijiang
提交者:
Piotr Nowojski
4月 17, 2020
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-16587][checkpointing] Spill the in-flight input and output buffers during checkpointing.
上级
f661d187
变更
18
显示空白变更内容
内联
并排
Showing
18 changed file
with
206 addition
and
8 deletion
+206
-8
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
...link/streaming/runtime/io/CheckpointBarrierUnaligner.java
+19
-2
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
...che/flink/streaming/runtime/io/CheckpointedInputGate.java
+4
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
...apache/flink/streaming/runtime/io/InputProcessorUtil.java
+7
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
...ache/flink/streaming/runtime/io/StreamInputProcessor.java
+5
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
...nk/streaming/runtime/io/StreamMultipleInputProcessor.java
+18
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java
...e/flink/streaming/runtime/io/StreamOneInputProcessor.java
+8
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskInput.java
...rg/apache/flink/streaming/runtime/io/StreamTaskInput.java
+8
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
...he/flink/streaming/runtime/io/StreamTaskNetworkInput.java
+26
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskSourceInput.java
...che/flink/streaming/runtime/io/StreamTaskSourceInput.java
+8
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
...e/flink/streaming/runtime/io/StreamTwoInputProcessor.java
+10
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java
...link/streaming/runtime/tasks/AsyncCheckpointRunnable.java
+6
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java
...link/streaming/runtime/tasks/MultipleInputStreamTask.java
+1
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
...che/flink/streaming/runtime/tasks/OneInputStreamTask.java
+1
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
.../org/apache/flink/streaming/runtime/tasks/StreamTask.java
+17
-1
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
...aming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
+52
-5
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
...che/flink/streaming/runtime/tasks/TwoInputStreamTask.java
+1
-0
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java
...ink/streaming/runtime/tasks/LocalStateForwardingTest.java
+2
-0
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
.../apache/flink/streaming/runtime/tasks/StreamTaskTest.java
+13
-0
未找到文件。
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
浏览文件 @
5cebfb76
...
...
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.runtime.io;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.runtime.checkpoint.CheckpointException
;
import
org.apache.flink.runtime.checkpoint.CheckpointFailureReason
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.checkpoint.channel.InputChannelInfo
;
import
org.apache.flink.runtime.concurrent.FutureUtils
;
import
org.apache.flink.runtime.io.network.api.CancelCheckpointMarker
;
...
...
@@ -43,6 +44,8 @@ import java.util.concurrent.CompletableFuture;
import
java.util.function.Function
;
import
java.util.stream.IntStream
;
import
static
org
.
apache
.
flink
.
util
.
Preconditions
.
checkNotNull
;
/**
* {@link CheckpointBarrierUnaligner} is used for triggering checkpoint while reading the first barrier
* and keeping track of the number of received barriers and consumed barriers.
...
...
@@ -86,6 +89,7 @@ public class CheckpointBarrierUnaligner extends CheckpointBarrierHandler {
CheckpointBarrierUnaligner
(
int
[]
numberOfInputChannelsPerGate
,
ChannelStateWriter
channelStateWriter
,
String
taskName
,
AbstractInvokable
toNotifyOnCheckpoint
)
{
super
(
toNotifyOnCheckpoint
);
...
...
@@ -108,7 +112,7 @@ public class CheckpointBarrierUnaligner extends CheckpointBarrierHandler {
.
flatMap
(
Function
.
identity
())
.
toArray
(
InputChannelInfo
[]::
new
);
threadSafeUnaligner
=
new
ThreadSafeUnaligner
(
totalNumChannels
,
this
);
threadSafeUnaligner
=
new
ThreadSafeUnaligner
(
totalNumChannels
,
checkNotNull
(
channelStateWriter
),
this
);
}
@Override
...
...
@@ -315,12 +319,16 @@ public class CheckpointBarrierUnaligner extends CheckpointBarrierHandler {
/** The number of opened channels. */
private
int
numOpenChannels
;
private
final
ChannelStateWriter
channelStateWriter
;
private
final
CheckpointBarrierUnaligner
handler
;
public
ThreadSafeUnaligner
(
int
totalNumChannels
,
ChannelStateWriter
channelStateWriter
,
CheckpointBarrierUnaligner
handler
)
{
storeNewBuffers
=
new
boolean
[
totalNumChannels
];
this
.
channelStateWriter
=
channelStateWriter
;
this
.
handler
=
handler
;
numOpenChannels
=
totalNumChannels
;
}
...
...
@@ -350,8 +358,16 @@ public class CheckpointBarrierUnaligner extends CheckpointBarrierHandler {
@Override
public
synchronized
void
notifyBufferReceived
(
Buffer
buffer
,
InputChannelInfo
channelInfo
)
{
if
(
storeNewBuffers
[
handler
.
getFlattenedChannelIndex
(
channelInfo
)])
{
channelStateWriter
.
addInputData
(
currentReceivedCheckpointId
,
channelInfo
,
ChannelStateWriter
.
SEQUENCE_NUMBER_UNKNOWN
,
buffer
);
}
else
{
buffer
.
recycleBuffer
();
}
}
@Override
public
synchronized
void
close
()
throws
IOException
{
...
...
@@ -382,6 +398,7 @@ public class CheckpointBarrierUnaligner extends CheckpointBarrierHandler {
Arrays
.
fill
(
storeNewBuffers
,
true
);
numBarriersReceived
=
0
;
allBarriersReceivedFuture
=
new
CompletableFuture
<>();
channelStateWriter
.
start
(
barrierId
,
barrier
.
getCheckpointOptions
());
}
public
synchronized
void
resetReceivedBarriers
(
long
checkpointId
)
{
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
浏览文件 @
5cebfb76
...
...
@@ -184,6 +184,10 @@ public class CheckpointedInputGate implements PullingAsyncDataInput<BufferOrEven
return
Collections
.
emptyList
();
}
public
CompletableFuture
<
Void
>
getAllBarriersReceivedFuture
(
long
checkpointId
)
{
return
((
CheckpointBarrierUnaligner
)
barrierHandler
).
getAllBarriersReceivedFuture
(
checkpointId
);
}
private
int
offsetChannelIndex
(
int
channelIndex
)
{
return
channelIndex
+
channelIndexOffset
;
}
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
浏览文件 @
5cebfb76
...
...
@@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal;
import
org.apache.flink.configuration.Configuration
;
import
org.apache.flink.configuration.IllegalConfigurationException
;
import
org.apache.flink.configuration.TaskManagerOptions
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.io.network.partition.consumer.InputGate
;
import
org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable
;
import
org.apache.flink.runtime.metrics.MetricNames
;
...
...
@@ -41,6 +42,7 @@ public class InputProcessorUtil {
public
static
CheckpointedInputGate
createCheckpointedInputGate
(
AbstractInvokable
toNotifyOnCheckpoint
,
StreamConfig
config
,
ChannelStateWriter
channelStateWriter
,
InputGate
inputGate
,
Configuration
taskManagerConfig
,
TaskIOMetricGroup
taskIOMetricGroup
,
...
...
@@ -52,6 +54,7 @@ public class InputProcessorUtil {
CheckpointBarrierHandler
barrierHandler
=
createCheckpointBarrierHandler
(
config
,
IntStream
.
of
(
inputGate
.
getNumberOfInputChannels
()),
channelStateWriter
,
taskName
,
toNotifyOnCheckpoint
);
registerCheckpointMetrics
(
taskIOMetricGroup
,
barrierHandler
);
...
...
@@ -68,6 +71,7 @@ public class InputProcessorUtil {
public
static
CheckpointedInputGate
[]
createCheckpointedInputGatePair
(
AbstractInvokable
toNotifyOnCheckpoint
,
StreamConfig
config
,
ChannelStateWriter
channelStateWriter
,
Configuration
taskManagerConfig
,
TaskIOMetricGroup
taskIOMetricGroup
,
String
taskName
,
...
...
@@ -92,6 +96,7 @@ public class InputProcessorUtil {
CheckpointBarrierHandler
barrierHandler
=
createCheckpointBarrierHandler
(
config
,
Arrays
.
stream
(
inputGates
).
mapToInt
(
InputGate:
:
getNumberOfInputChannels
),
channelStateWriter
,
taskName
,
toNotifyOnCheckpoint
);
registerCheckpointMetrics
(
taskIOMetricGroup
,
barrierHandler
);
...
...
@@ -125,6 +130,7 @@ public class InputProcessorUtil {
private
static
CheckpointBarrierHandler
createCheckpointBarrierHandler
(
StreamConfig
config
,
IntStream
numberOfInputChannelsPerGate
,
ChannelStateWriter
channelStateWriter
,
String
taskName
,
AbstractInvokable
toNotifyOnCheckpoint
)
{
switch
(
config
.
getCheckpointMode
())
{
...
...
@@ -132,6 +138,7 @@ public class InputProcessorUtil {
if
(
config
.
isUnalignedCheckpointsEnabled
())
{
return
new
CheckpointBarrierUnaligner
(
numberOfInputChannelsPerGate
.
toArray
(),
channelStateWriter
,
taskName
,
toNotifyOnCheckpoint
);
}
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
浏览文件 @
5cebfb76
...
...
@@ -19,9 +19,12 @@
package
org.apache.flink.streaming.runtime.io
;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.io.AvailabilityProvider
;
import
java.io.Closeable
;
import
java.io.IOException
;
import
java.util.concurrent.CompletableFuture
;
/**
* Interface for processing records by {@link org.apache.flink.streaming.runtime.tasks.StreamTask}.
...
...
@@ -34,4 +37,6 @@ public interface StreamInputProcessor extends AvailabilityProvider, Closeable {
* state and/or {@link #getAvailableFuture()}.
*/
InputStatus
processInput
()
throws
Exception
;
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
throws
IOException
;
}
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
浏览文件 @
5cebfb76
...
...
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.runtime.io;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.api.common.typeutils.TypeSerializer
;
import
org.apache.flink.metrics.Counter
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.io.disk.iomanager.IOManager
;
import
org.apache.flink.streaming.api.operators.Input
;
import
org.apache.flink.streaming.api.operators.InputSelection
;
...
...
@@ -176,6 +177,17 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor
}
}
@Override
public
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
throws
IOException
{
CompletableFuture
<?>[]
inputFutures
=
new
CompletableFuture
[
inputProcessors
.
length
];
for
(
int
index
=
0
;
index
<
inputFutures
.
length
;
index
++)
{
inputFutures
[
index
]
=
inputProcessors
[
index
].
prepareSnapshot
(
channelStateWriter
,
checkpointId
);
}
return
CompletableFuture
.
allOf
(
inputFutures
);
}
private
int
selectNextReadingInputIndex
()
{
if
(!
inputSelectionHandler
.
isAnyInputAvailable
())
{
fullCheckAndSetAvailable
();
...
...
@@ -238,6 +250,12 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor
public
void
close
()
throws
IOException
{
networkInput
.
close
();
}
public
CompletableFuture
<?>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
throws
IOException
{
return
networkInput
.
prepareSnapshot
(
channelStateWriter
,
checkpointId
);
}
}
/**
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java
浏览文件 @
5cebfb76
...
...
@@ -19,6 +19,7 @@
package
org.apache.flink.streaming.runtime.io
;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput
;
import
org.apache.flink.streaming.runtime.tasks.OperatorChain
;
...
...
@@ -71,6 +72,13 @@ public final class StreamOneInputProcessor<IN> implements StreamInputProcessor {
return
status
;
}
@Override
public
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
throws
IOException
{
return
input
.
prepareSnapshot
(
channelStateWriter
,
checkpointId
);
}
@Override
public
void
close
()
throws
IOException
{
input
.
close
();
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskInput.java
浏览文件 @
5cebfb76
...
...
@@ -18,8 +18,11 @@
package
org.apache.flink.streaming.runtime.io
;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
java.io.Closeable
;
import
java.io.IOException
;
import
java.util.concurrent.CompletableFuture
;
/**
* Basic interface for inputs of stream operators.
...
...
@@ -32,4 +35,9 @@ public interface StreamTaskInput<T> extends PushingAsyncDataInput<T>, Closeable
* Returns the input index of this input.
*/
int
getInputIndex
();
/**
* Prepares to spill the in-flight input buffers as checkpoint snapshot.
*/
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
throws
IOException
;
}
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
浏览文件 @
5cebfb76
...
...
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.runtime.io;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.annotation.VisibleForTesting
;
import
org.apache.flink.api.common.typeutils.TypeSerializer
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.event.AbstractEvent
;
import
org.apache.flink.runtime.io.disk.iomanager.IOManager
;
import
org.apache.flink.runtime.io.network.api.EndOfPartitionEvent
;
...
...
@@ -29,6 +30,7 @@ import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.
import
org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer
;
import
org.apache.flink.runtime.io.network.buffer.Buffer
;
import
org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent
;
import
org.apache.flink.runtime.io.network.partition.consumer.InputChannel
;
import
org.apache.flink.runtime.plugable.DeserializationDelegate
;
import
org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate
;
import
org.apache.flink.streaming.api.watermark.Watermark
;
...
...
@@ -197,6 +199,30 @@ public final class StreamTaskNetworkInput<T> implements StreamTaskInput<T> {
return
checkpointedInputGate
.
getAvailableFuture
();
}
@Override
public
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
throws
IOException
{
for
(
int
channelIndex
=
0
;
channelIndex
<
recordDeserializers
.
length
;
channelIndex
++)
{
final
InputChannel
channel
=
checkpointedInputGate
.
getChannel
(
channelIndex
);
// Assumption for retrieving buffers = one concurrent checkpoint
recordDeserializers
[
channelIndex
].
getUnconsumedBuffer
().
ifPresent
(
buffer
->
channelStateWriter
.
addInputData
(
checkpointId
,
channel
.
getChannelInfo
(),
ChannelStateWriter
.
SEQUENCE_NUMBER_UNKNOWN
,
buffer
));
channelStateWriter
.
addInputData
(
checkpointId
,
channel
.
getChannelInfo
(),
ChannelStateWriter
.
SEQUENCE_NUMBER_UNKNOWN
,
checkpointedInputGate
.
requestInflightBuffers
(
checkpointId
,
channelIndex
).
toArray
(
new
Buffer
[
0
]));
}
return
checkpointedInputGate
.
getAllBarriersReceivedFuture
(
checkpointId
);
}
@Override
public
void
close
()
throws
IOException
{
// release the deserializers . this part should not ever fail
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskSourceInput.java
浏览文件 @
5cebfb76
...
...
@@ -19,6 +19,7 @@
package
org.apache.flink.streaming.runtime.io
;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.streaming.api.operators.SourceReaderOperator
;
import
org.apache.flink.util.IOUtils
;
...
...
@@ -62,5 +63,12 @@ public final class StreamTaskSourceInput<T> implements StreamTaskInput<T> {
public
void
close
()
{
IOUtils
.
closeQuietly
(
operator:
:
close
);
}
@Override
public
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
{
return
CompletableFuture
.
completedFuture
(
null
);
}
}
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
浏览文件 @
5cebfb76
...
...
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.runtime.io;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.api.common.typeutils.TypeSerializer
;
import
org.apache.flink.metrics.Counter
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.io.disk.iomanager.IOManager
;
import
org.apache.flink.streaming.api.operators.InputSelection
;
import
org.apache.flink.streaming.api.operators.TwoInputStreamOperator
;
...
...
@@ -184,6 +185,15 @@ public final class StreamTwoInputProcessor<IN1, IN2> implements StreamInputProce
return
getInputStatus
();
}
@Override
public
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
throws
IOException
{
return
CompletableFuture
.
allOf
(
input1
.
prepareSnapshot
(
channelStateWriter
,
checkpointId
),
input2
.
prepareSnapshot
(
channelStateWriter
,
checkpointId
));
}
private
int
selectFirstReadingInputIndex
()
throws
IOException
{
// Note: the first call to nextSelection () on the operator must be made after this operator
// is opened to ensure that any changes about the input selection in its open()
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java
浏览文件 @
5cebfb76
...
...
@@ -34,6 +34,7 @@ import org.slf4j.LoggerFactory;
import
java.io.Closeable
;
import
java.util.Map
;
import
java.util.concurrent.Future
;
import
java.util.concurrent.atomic.AtomicReference
;
import
static
org
.
apache
.
flink
.
util
.
Preconditions
.
checkNotNull
;
...
...
@@ -58,6 +59,7 @@ final class AsyncCheckpointRunnable implements Runnable, Closeable {
private
final
Map
<
OperatorID
,
OperatorSnapshotFutures
>
operatorSnapshotsInProgress
;
private
final
CheckpointMetaData
checkpointMetaData
;
private
final
CheckpointMetrics
checkpointMetrics
;
private
final
Future
<?>
channelWrittenFuture
;
private
final
long
asyncStartNanos
;
private
final
AtomicReference
<
AsyncCheckpointState
>
asyncCheckpointState
=
new
AtomicReference
<>(
AsyncCheckpointState
.
RUNNING
);
...
...
@@ -65,6 +67,7 @@ final class AsyncCheckpointRunnable implements Runnable, Closeable {
Map
<
OperatorID
,
OperatorSnapshotFutures
>
operatorSnapshotsInProgress
,
CheckpointMetaData
checkpointMetaData
,
CheckpointMetrics
checkpointMetrics
,
Future
<?>
channelWrittenFuture
,
long
asyncStartNanos
,
String
taskName
,
CloseableRegistry
closeableRegistry
,
...
...
@@ -74,6 +77,7 @@ final class AsyncCheckpointRunnable implements Runnable, Closeable {
this
.
operatorSnapshotsInProgress
=
checkNotNull
(
operatorSnapshotsInProgress
);
this
.
checkpointMetaData
=
checkNotNull
(
checkpointMetaData
);
this
.
checkpointMetrics
=
checkNotNull
(
checkpointMetrics
);
this
.
channelWrittenFuture
=
checkNotNull
(
channelWrittenFuture
);
this
.
asyncStartNanos
=
asyncStartNanos
;
this
.
taskName
=
checkNotNull
(
taskName
);
this
.
closeableRegistry
=
checkNotNull
(
closeableRegistry
);
...
...
@@ -113,6 +117,8 @@ final class AsyncCheckpointRunnable implements Runnable, Closeable {
checkpointMetrics
.
setAsyncDurationMillis
(
asyncDurationMillis
);
channelWrittenFuture
.
get
();
if
(
asyncCheckpointState
.
compareAndSet
(
AsyncCheckpointState
.
RUNNING
,
AsyncCheckpointState
.
COMPLETED
))
{
reportCompletedSnapshotStates
(
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java
浏览文件 @
5cebfb76
...
...
@@ -101,6 +101,7 @@ public class MultipleInputStreamTask<OUT> extends StreamTask<OUT, MultipleInputS
CheckpointedInputGate
[]
checkpointedInputGates
=
InputProcessorUtil
.
createCheckpointedInputGatePair
(
this
,
getConfiguration
(),
getChannelStateWriter
(),
getEnvironment
().
getTaskManagerInfo
().
getConfiguration
(),
getEnvironment
().
getMetricGroup
().
getIOMetricGroup
(),
getTaskNameWithSubtaskAndId
(),
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
浏览文件 @
5cebfb76
...
...
@@ -107,6 +107,7 @@ public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, OneInputStreamO
return
InputProcessorUtil
.
createCheckpointedInputGate
(
this
,
configuration
,
getChannelStateWriter
(),
inputGate
,
getEnvironment
().
getTaskManagerInfo
().
getConfiguration
(),
getEnvironment
().
getMetricGroup
().
getIOMetricGroup
(),
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
浏览文件 @
5cebfb76
...
...
@@ -29,6 +29,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
import
org.apache.flink.runtime.checkpoint.CheckpointMetrics
;
import
org.apache.flink.runtime.checkpoint.CheckpointOptions
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.concurrent.FutureUtils
;
import
org.apache.flink.runtime.execution.CancelTaskException
;
import
org.apache.flink.runtime.execution.Environment
;
...
...
@@ -80,6 +81,7 @@ import org.slf4j.LoggerFactory;
import
javax.annotation.Nullable
;
import
java.io.IOException
;
import
java.util.ArrayList
;
import
java.util.List
;
import
java.util.Map
;
...
...
@@ -281,7 +283,8 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
getAsyncOperationsThreadPool
(),
getEnvironment
(),
this
,
false
);
// todo: pass true if unaligned checkpoints enabled
configuration
.
isUnalignedCheckpointsEnabled
(),
this
::
prepareInputSnapshot
);
// if the clock is not already set, then assign a default TimeServiceProvider
if
(
timerService
==
null
)
{
...
...
@@ -292,6 +295,17 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
}
}
private
CompletableFuture
<
Void
>
prepareInputSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
throws
IOException
{
if
(
inputProcessor
==
null
)
{
return
FutureUtils
.
completedVoidFuture
();
}
return
inputProcessor
.
prepareSnapshot
(
channelStateWriter
,
checkpointId
);
}
protected
ChannelStateWriter
getChannelStateWriter
()
{
return
subtaskCheckpointCoordinator
.
getChannelStateWriter
();
}
// ------------------------------------------------------------------------
// Life cycle methods for specific implementations
// ------------------------------------------------------------------------
...
...
@@ -726,6 +740,7 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
.
setBytesBufferedInAlignment
(
0L
)
.
setAlignmentDurationNanos
(
0L
);
subtaskCheckpointCoordinator
.
getChannelStateWriter
().
start
(
checkpointMetaData
.
getCheckpointId
(),
checkpointOptions
);
boolean
success
=
performCheckpoint
(
checkpointMetaData
,
checkpointOptions
,
checkpointMetrics
,
advanceToEndOfEventTime
);
if
(!
success
)
{
declineCheckpoint
(
checkpointMetaData
.
getCheckpointId
());
...
...
@@ -865,6 +880,7 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
LOG
.
debug
(
"Ignoring notification of complete checkpoint for not-running task {}"
,
getName
());
}
subtaskCheckpointCoordinator
.
getChannelStateWriter
().
notifyCheckpointComplete
(
checkpointId
);
getEnvironment
().
getTaskStateManager
().
notifyCheckpointComplete
(
checkpointId
);
if
(
isRunning
&&
isSynchronousSavepointId
(
checkpointId
))
{
finishTask
();
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
浏览文件 @
5cebfb76
...
...
@@ -25,9 +25,13 @@ import org.apache.flink.runtime.checkpoint.StateObjectCollection;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriterImpl
;
import
org.apache.flink.runtime.concurrent.FutureUtils
;
import
org.apache.flink.runtime.execution.Environment
;
import
org.apache.flink.runtime.io.network.api.CancelCheckpointMarker
;
import
org.apache.flink.runtime.io.network.api.CheckpointBarrier
;
import
org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter
;
import
org.apache.flink.runtime.io.network.buffer.Buffer
;
import
org.apache.flink.runtime.io.network.partition.ResultSubpartition
;
import
org.apache.flink.runtime.jobgraph.OperatorID
;
import
org.apache.flink.runtime.state.CheckpointStorageLocationReference
;
import
org.apache.flink.runtime.state.CheckpointStorageWorkerView
;
...
...
@@ -36,6 +40,7 @@ import org.apache.flink.runtime.state.SnapshotResult;
import
org.apache.flink.streaming.api.operators.OperatorSnapshotFutures
;
import
org.apache.flink.streaming.api.operators.StreamOperator
;
import
org.apache.flink.util.FlinkRuntimeException
;
import
org.apache.flink.util.function.BiFunctionWithException
;
import
org.slf4j.Logger
;
import
org.slf4j.LoggerFactory
;
...
...
@@ -43,8 +48,10 @@ import org.slf4j.LoggerFactory;
import
java.io.IOException
;
import
java.util.HashMap
;
import
java.util.Map
;
import
java.util.concurrent.CompletableFuture
;
import
java.util.concurrent.ConcurrentHashMap
;
import
java.util.concurrent.ExecutorService
;
import
java.util.concurrent.Future
;
import
java.util.function.Supplier
;
import
static
org
.
apache
.
flink
.
runtime
.
checkpoint
.
CheckpointType
.
CHECKPOINT
;
...
...
@@ -62,6 +69,8 @@ class SubtaskCheckpointCoordinatorImpl implements SubtaskCheckpointCoordinator {
private
final
AsyncExceptionHandler
asyncExceptionHandler
;
private
final
ChannelStateWriter
channelStateWriter
;
private
final
StreamTaskActionExecutor
actionExecutor
;
private
final
boolean
unalignedCheckpointEnabled
;
private
final
BiFunctionWithException
<
ChannelStateWriter
,
Long
,
CompletableFuture
<
Void
>,
IOException
>
prepareInputSnapshot
;
SubtaskCheckpointCoordinatorImpl
(
CheckpointStorageWorkerView
checkpointStorage
,
...
...
@@ -71,7 +80,8 @@ class SubtaskCheckpointCoordinatorImpl implements SubtaskCheckpointCoordinator {
ExecutorService
executorService
,
Environment
env
,
AsyncExceptionHandler
asyncExceptionHandler
,
boolean
sendChannelState
)
throws
IOException
{
boolean
unalignedCheckpointEnabled
,
BiFunctionWithException
<
ChannelStateWriter
,
Long
,
CompletableFuture
<
Void
>,
IOException
>
prepareInputSnapshot
)
throws
IOException
{
this
.
checkpointStorage
=
new
CachingCheckpointStorageWorkerView
(
checkNotNull
(
checkpointStorage
));
this
.
taskName
=
checkNotNull
(
taskName
);
this
.
closeableRegistry
=
checkNotNull
(
closeableRegistry
);
...
...
@@ -79,11 +89,13 @@ class SubtaskCheckpointCoordinatorImpl implements SubtaskCheckpointCoordinator {
this
.
env
=
checkNotNull
(
env
);
this
.
asyncExceptionHandler
=
checkNotNull
(
asyncExceptionHandler
);
this
.
actionExecutor
=
checkNotNull
(
actionExecutor
);
this
.
channelStateWriter
=
sendChannelState
?
openChannelStateWriter
()
:
ChannelStateWriter
.
NO_OP
;
this
.
channelStateWriter
=
unalignedCheckpointEnabled
?
openChannelStateWriter
()
:
ChannelStateWriter
.
NO_OP
;
this
.
unalignedCheckpointEnabled
=
unalignedCheckpointEnabled
;
this
.
prepareInputSnapshot
=
prepareInputSnapshot
;
this
.
closeableRegistry
.
registerCloseable
(
this
);
}
private
ChannelStateWriter
Impl
openChannelStateWriter
()
{
private
ChannelStateWriter
openChannelStateWriter
()
{
ChannelStateWriterImpl
writer
=
new
ChannelStateWriterImpl
(
this
.
checkpointStorage
);
writer
.
open
();
return
writer
;
...
...
@@ -136,9 +148,16 @@ class SubtaskCheckpointCoordinatorImpl implements SubtaskCheckpointCoordinator {
// Step (2): Send the checkpoint barrier downstream
operatorChain
.
broadcastEvent
(
new
CheckpointBarrier
(
metadata
.
getCheckpointId
(),
metadata
.
getTimestamp
(),
options
));
new
CheckpointBarrier
(
metadata
.
getCheckpointId
(),
metadata
.
getTimestamp
(),
options
),
unalignedCheckpointEnabled
);
// Step (3): Take the state snapshot. This should be largely asynchronous, to not impact progress of the streaming topology
// Step (3): Prepare to spill the in-flight buffers for input and output
if
(
unalignedCheckpointEnabled
)
{
prepareInflightDataSnapshot
(
metadata
.
getCheckpointId
());
}
// Step (4): Take the state snapshot. This should be largely asynchronous, to not impact progress of the
// streaming topology
Map
<
OperatorID
,
OperatorSnapshotFutures
>
snapshotFutures
=
new
HashMap
<>(
operatorChain
.
getNumberOfOperators
());
try
{
...
...
@@ -184,12 +203,40 @@ class SubtaskCheckpointCoordinatorImpl implements SubtaskCheckpointCoordinator {
}
}
private
void
prepareInflightDataSnapshot
(
long
checkpointId
)
throws
IOException
{
prepareInputSnapshot
.
apply
(
channelStateWriter
,
checkpointId
)
.
thenAccept
(
unused
->
channelStateWriter
.
finishInput
(
checkpointId
));
ResultPartitionWriter
[]
writers
=
env
.
getAllWriters
();
for
(
ResultPartitionWriter
writer
:
writers
)
{
for
(
int
i
=
0
;
i
<
writer
.
getNumberOfSubpartitions
();
i
++)
{
ResultSubpartition
subpartition
=
writer
.
getSubpartition
(
i
);
channelStateWriter
.
addOutputData
(
checkpointId
,
subpartition
.
getSubpartitionInfo
(),
ChannelStateWriter
.
SEQUENCE_NUMBER_UNKNOWN
,
subpartition
.
requestInflightBufferSnapshot
().
toArray
(
new
Buffer
[
0
]));
}
}
channelStateWriter
.
finishOutput
(
checkpointId
);
}
private
void
finishAndReportAsync
(
Map
<
OperatorID
,
OperatorSnapshotFutures
>
snapshotFutures
,
CheckpointMetaData
metadata
,
CheckpointMetrics
metrics
)
{
final
Future
<?>
channelWrittenFuture
;
if
(
unalignedCheckpointEnabled
)
{
ChannelStateWriteResult
writeResult
=
channelStateWriter
.
getWriteResult
(
metadata
.
getCheckpointId
());
channelWrittenFuture
=
CompletableFuture
.
allOf
(
writeResult
.
getInputChannelStateHandles
(),
writeResult
.
getResultSubpartitionStateHandles
());
}
else
{
channelWrittenFuture
=
FutureUtils
.
completedVoidFuture
();
}
// we are transferring ownership over snapshotInProgressList for cleanup to the thread, active on submit
executorService
.
execute
(
new
AsyncCheckpointRunnable
(
snapshotFutures
,
metadata
,
metrics
,
channelWrittenFuture
,
System
.
nanoTime
(),
taskName
,
closeableRegistry
,
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
浏览文件 @
5cebfb76
...
...
@@ -62,6 +62,7 @@ public class TwoInputStreamTask<IN1, IN2, OUT> extends AbstractTwoInputStreamTas
CheckpointedInputGate
[]
checkpointedInputGates
=
InputProcessorUtil
.
createCheckpointedInputGatePair
(
this
,
getConfiguration
(),
getChannelStateWriter
(),
getEnvironment
().
getTaskManagerInfo
().
getConfiguration
(),
getEnvironment
().
getMetricGroup
().
getIOMetricGroup
(),
getTaskNameWithSubtaskAndId
(),
...
...
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java
浏览文件 @
5cebfb76
...
...
@@ -59,6 +59,7 @@ import javax.annotation.Nullable;
import
java.util.HashMap
;
import
java.util.Map
;
import
java.util.concurrent.CompletableFuture
;
import
java.util.concurrent.Executor
;
import
java.util.concurrent.Future
;
import
java.util.concurrent.RunnableFuture
;
...
...
@@ -114,6 +115,7 @@ public class LocalStateForwardingTest extends TestLogger {
snapshots
,
checkpointMetaData
,
checkpointMetrics
,
CompletableFuture
.
completedFuture
(
null
),
0L
,
testStreamTask
.
getName
(),
testStreamTask
.
getCancelables
(),
...
...
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
浏览文件 @
5cebfb76
...
...
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
import
org.apache.flink.runtime.checkpoint.SubtaskState
;
import
org.apache.flink.runtime.checkpoint.TaskStateSnapshot
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader
;
import
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter
;
import
org.apache.flink.runtime.concurrent.FutureUtils
;
import
org.apache.flink.runtime.concurrent.TestingUncaughtExceptionHandler
;
import
org.apache.flink.runtime.execution.CancelTaskException
;
...
...
@@ -1173,6 +1174,13 @@ public class StreamTaskTest extends TestLogger {
return
++
currentNumProcessCalls
<
totalProcessCalls
?
InputStatus
.
MORE_AVAILABLE
:
InputStatus
.
END_OF_INPUT
;
}
@Override
public
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
final
long
checkpointId
)
{
return
FutureUtils
.
completedVoidFuture
();
}
@Override
public
void
close
()
throws
IOException
{
}
...
...
@@ -1338,6 +1346,11 @@ public class StreamTaskTest extends TestLogger {
return
isFinished
?
InputStatus
.
END_OF_INPUT
:
InputStatus
.
NOTHING_AVAILABLE
;
}
@Override
public
CompletableFuture
<
Void
>
prepareSnapshot
(
ChannelStateWriter
channelStateWriter
,
long
checkpointId
)
{
return
FutureUtils
.
completedVoidFuture
();
}
@Override
public
void
close
()
throws
IOException
{
}
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录