Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 9e347ea

Browse files
committed
[SHOPEE-241][checkpoint] Merge the channel state files
1 parent 37782c0 commit 9e347ea

30 files changed

+2073
-178
lines changed

docs/layouts/shortcodes/generated/execution_checkpointing_configuration.html

+6
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,12 @@
2020
<td>Boolean</td>
2121
<td>Feature toggle for enabling checkpointing even if some of tasks have finished. Before you enable it, please take a look at <a href="{{.Site.BaseURL}}{{.Site.LanguagePrefix}}/docs/dev/datastream/fault-tolerance/checkpointing/#checkpointing-with-parts-of-the-graph-finished-beta">the important considerations</a> </td>
2222
</tr>
23+
<tr>
24+
<td><h5>execution.checkpointing.channel-state.number-of-tasks-share-file</h5></td>
25+
<td style="word-wrap: break-word;">5</td>
26+
<td>Integer</td>
27+
<td>Defines the maximum number of tasks that share the same channel state file. It can reduce the number of small files when enable unaligned checkpoint. Each task will create a new channel state file when this is configured to 1.</td>
28+
</tr>
2329
<tr>
2430
<td><h5>execution.checkpointing.externalized-checkpoint-retention</h5></td>
2531
<td style="word-wrap: break-word;">NO_EXTERNALIZED_CHECKPOINTS</td>

flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java

+1
Original file line numberDiff line numberDiff line change
@@ -224,6 +224,7 @@ public void checkFailureCounter(CheckpointException exception, long checkpointId
224224
case CHECKPOINT_SUBSUMED:
225225
case CHECKPOINT_COORDINATOR_SUSPEND:
226226
case CHECKPOINT_COORDINATOR_SHUTDOWN:
227+
case CHANNEL_STATE_SHARED_STREAM_EXCEPTION:
227228
case JOB_FAILURE:
228229
case JOB_FAILOVER_REGION:
229230
// for compatibility purposes with user job behavior

flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureReason.java

+3
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,9 @@ public enum CheckpointFailureReason {
3939

4040
CHECKPOINT_ASYNC_EXCEPTION(false, "Asynchronous task checkpoint failed."),
4141

42+
CHANNEL_STATE_SHARED_STREAM_EXCEPTION(
43+
false, "The checkpoint is aborted due to other Task exceptions."),
44+
4245
CHECKPOINT_EXPIRED(false, "Checkpoint expired before completing."),
4346

4447
CHECKPOINT_SUBSUMED(false, "Checkpoint has been subsumed."),
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,76 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.flink.runtime.checkpoint.channel;
19+
20+
import org.apache.flink.api.common.JobID;
21+
import org.apache.flink.runtime.jobgraph.JobVertexID;
22+
import org.apache.flink.runtime.state.CheckpointStateOutputStream;
23+
import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
24+
import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
25+
import org.apache.flink.runtime.state.StreamStateHandle;
26+
27+
import java.io.IOException;
28+
29+
/**
30+
* The manager of ChannelStateCheckpointStream. Its responsibilities are as follows: 1. Create a new
31+
* output stream or get an old output stream for each task 2. Close the channel state output stream
32+
* and callback the success method of all tasks when all shared tasks completed checkpoint 3. Close
33+
* the channel state output stream and callback the fail method of all tasks when any tasks failed
34+
* checkpoint.
35+
*/
36+
public interface ChannelStateCheckpointStreamManager {
37+
38+
void registerTask(JobID jobID, JobVertexID jobVertexID, int subtaskIndex);
39+
40+
void unregisterTask(JobID jobID, JobVertexID jobVertexID, int subtaskIndex);
41+
42+
/** Create or get a channel state output stream, and register callback. */
43+
CheckpointStateOutputStream getOutputStream(
44+
JobID jobID,
45+
JobVertexID jobVertexID,
46+
int subtaskIndex,
47+
long checkpointId,
48+
CheckpointStorageLocationReference locationReference,
49+
CheckpointStorageWorkerView streamFactoryResolver,
50+
ChannelStateCallback callback)
51+
throws IOException;
52+
53+
void completeCheckpoint(
54+
JobID jobID, long checkpointId, JobVertexID jobVertexID, long subtaskIndex)
55+
throws IOException;
56+
57+
void failCheckpoint(
58+
JobID jobID,
59+
long checkpointId,
60+
JobVertexID jobVertexID,
61+
long subtaskIndex,
62+
Throwable e);
63+
64+
/** The channel state callback. */
65+
interface ChannelStateCallback {
66+
67+
/** It will be called when the channel state file has been written. */
68+
void success(StreamStateHandle stateHandle) throws IOException;
69+
70+
/**
71+
* It will be called when any problems are encountered during the writing of the channel
72+
* state file.
73+
*/
74+
void fail(Throwable e);
75+
}
76+
}

0 commit comments

Comments
 (0)