/
ZooKeeperCompletedCheckpointStoreITCase.java
195 lines (151 loc) · 6.22 KB
/
ZooKeeperCompletedCheckpointStoreITCase.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
/*
* 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.runtime.checkpoint;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.runtime.state.RetrievableStateHandle;
import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper;
import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.Test;
import scala.concurrent.duration.Deadline;
import scala.concurrent.duration.FiniteDuration;
import java.io.IOException;
import java.io.Serializable;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
/**
* Tests for basic {@link CompletedCheckpointStore} contract and ZooKeeper state handling.
*/
public class ZooKeeperCompletedCheckpointStoreITCase extends CompletedCheckpointStoreTest {
private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
private final static String CheckpointsPath = "/checkpoints";
@AfterClass
public static void tearDown() throws Exception {
if (ZooKeeper != null) {
ZooKeeper.shutdown();
}
}
@Before
public void cleanUp() throws Exception {
ZooKeeper.deleteAll();
}
@Override
protected CompletedCheckpointStore createCompletedCheckpoints(
int maxNumberOfCheckpointsToRetain, ClassLoader userLoader) throws Exception {
return new ZooKeeperCompletedCheckpointStore(maxNumberOfCheckpointsToRetain, userLoader,
ZooKeeper.createClient(), CheckpointsPath, new RetrievableStateStorageHelper<CompletedCheckpoint>() {
@Override
public RetrievableStateHandle<CompletedCheckpoint> store(CompletedCheckpoint state) throws Exception {
return new HeapRetrievableStateHandle<CompletedCheckpoint>(state);
}
});
}
// ---------------------------------------------------------------------------------------------
/**
* Tests that older checkpoints are cleaned up at startup.
*/
@Test
public void testRecover() throws Exception {
CompletedCheckpointStore checkpoints = createCompletedCheckpoints(3, ClassLoader
.getSystemClassLoader());
TestCompletedCheckpoint[] expected = new TestCompletedCheckpoint[] {
createCheckpoint(0), createCheckpoint(1), createCheckpoint(2)
};
// Add multiple checkpoints
checkpoints.addCheckpoint(expected[0]);
checkpoints.addCheckpoint(expected[1]);
checkpoints.addCheckpoint(expected[2]);
// All three should be in ZK
assertEquals(3, ZooKeeper.getClient().getChildren().forPath(CheckpointsPath).size());
assertEquals(3, checkpoints.getNumberOfRetainedCheckpoints());
// Recover
checkpoints.recover();
// Only the latest one should be in ZK
Deadline deadline = new FiniteDuration(1, TimeUnit.MINUTES).fromNow();
// Retry this operation, because removal is asynchronous
while (deadline.hasTimeLeft() && ZooKeeper.getClient()
.getChildren().forPath(CheckpointsPath).size() != 1) {
Thread.sleep(Math.min(100, deadline.timeLeft().toMillis()));
}
assertEquals(1, ZooKeeper.getClient().getChildren().forPath(CheckpointsPath).size());
assertEquals(1, checkpoints.getNumberOfRetainedCheckpoints());
assertEquals(expected[2], checkpoints.getLatestCheckpoint());
}
/**
* Tests that shutdown discards all checkpoints.
*/
@Test
public void testShutdownDiscardsCheckpoints() throws Exception {
CuratorFramework client = ZooKeeper.getClient();
CompletedCheckpointStore store = createCompletedCheckpoints(1, ClassLoader.getSystemClassLoader());
TestCompletedCheckpoint checkpoint = createCheckpoint(0);
store.addCheckpoint(checkpoint);
assertEquals(1, store.getNumberOfRetainedCheckpoints());
assertNotNull(client.checkExists().forPath(CheckpointsPath + "/" + checkpoint.getCheckpointID()));
store.shutdown();
assertEquals(0, store.getNumberOfRetainedCheckpoints());
assertNull(client.checkExists().forPath(CheckpointsPath + "/" + checkpoint.getCheckpointID()));
store.recover();
assertEquals(0, store.getNumberOfRetainedCheckpoints());
}
/**
* Tests that suspends keeps all checkpoints (as they can be recovered
* later by the ZooKeeper store).
*/
@Test
public void testSuspendKeepsCheckpoints() throws Exception {
CuratorFramework client = ZooKeeper.getClient();
CompletedCheckpointStore store = createCompletedCheckpoints(1, ClassLoader.getSystemClassLoader());
TestCompletedCheckpoint checkpoint = createCheckpoint(0);
store.addCheckpoint(checkpoint);
assertEquals(1, store.getNumberOfRetainedCheckpoints());
assertNotNull(client.checkExists().forPath(CheckpointsPath + "/" + checkpoint.getCheckpointID()));
store.suspend();
assertEquals(0, store.getNumberOfRetainedCheckpoints());
assertNotNull(client.checkExists().forPath(CheckpointsPath + "/" + checkpoint.getCheckpointID()));
// Recover again
store.recover();
CompletedCheckpoint recovered = store.getLatestCheckpoint();
assertEquals(checkpoint, recovered);
}
static class HeapRetrievableStateHandle<T extends Serializable> implements RetrievableStateHandle<T> {
private static final long serialVersionUID = -268548467968932L;
public HeapRetrievableStateHandle(T state) {
this.state = state;
}
private T state;
@Override
public T retrieveState() throws Exception {
return state;
}
@Override
public void discardState() throws Exception {
state = null;
}
@Override
public long getStateSize() throws IOException {
return 0;
}
@Override
public void close() throws IOException {
}
}
}