forked from apache/asterixdb
/
RecoveryTask.java
194 lines (184 loc) · 8.65 KB
/
RecoveryTask.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
/*
* 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.asterix.app.active;
import java.util.concurrent.Callable;
import org.apache.asterix.active.ActivityState;
import org.apache.asterix.active.IRetryPolicyFactory;
import org.apache.asterix.active.NoRetryPolicyFactory;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
import org.apache.asterix.common.api.IMetadataLockManager;
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.metadata.utils.MetadataLockUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.util.IRetryPolicy;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
public class RecoveryTask {
private static final Logger LOGGER = LogManager.getLogger();
private static final Level level = Level.INFO;
private final ActiveEntityEventsListener listener;
private volatile boolean cancelRecovery = false;
private final IRetryPolicyFactory retryPolicyFactory;
private final MetadataProvider metadataProvider;
private final IClusterStateManager clusterStateManager;
private Exception failure;
public RecoveryTask(ICcApplicationContext appCtx, ActiveEntityEventsListener listener,
IRetryPolicyFactory retryPolicyFactory) {
this.listener = listener;
this.retryPolicyFactory = retryPolicyFactory;
this.metadataProvider = new MetadataProvider(appCtx, null);
this.clusterStateManager = appCtx.getClusterStateManager();
}
public Callable<Void> recover() {
if (retryPolicyFactory == NoRetryPolicyFactory.INSTANCE) {
return () -> null;
}
IRetryPolicy policy = retryPolicyFactory.create(listener);
return () -> {
Thread.currentThread().setName("RecoveryTask (" + listener.getEntityId() + ")");
doRecover(policy);
return null;
};
}
public void cancel() {
cancelRecovery = true;
}
protected void resumeOrRecover(MetadataProvider metadataProvider) throws HyracksDataException {
try {
synchronized (listener) {
listener.doResume(metadataProvider);
listener.setState(ActivityState.RUNNING);
}
} catch (Exception e) {
LOGGER.log(Level.WARN, "Attempt to resume " + listener.getEntityId() + " Failed", e);
synchronized (listener) {
if (listener.getState() == ActivityState.RESUMING) {
// This will be the case if compilation failure
// If the failure is a runtime failure, then the state
// would've been set to temporarily failed already
listener.setState(ActivityState.TEMPORARILY_FAILED);
}
}
if (retryPolicyFactory == NoRetryPolicyFactory.INSTANCE) {
synchronized (listener) {
if (!cancelRecovery) {
listener.setState(ActivityState.STOPPED);
listener.setRunning(metadataProvider, false);
}
}
} else {
LOGGER.log(Level.WARN, "Submitting recovery task for " + listener.getEntityId());
metadataProvider.getApplicationContext().getServiceContext().getControllerService().getExecutor()
.submit(() -> doRecover(retryPolicyFactory.create(listener)));
}
throw e;
}
}
protected Void doRecover(IRetryPolicy policy)
throws AlgebricksException, HyracksDataException, InterruptedException {
LOGGER.log(level, "Actual Recovery task has started");
if (listener.getState() != ActivityState.TEMPORARILY_FAILED) {
LOGGER.log(level, "but its state is not temp failure and so we're just returning");
return null;
}
LOGGER.log(level, "calling the policy");
while (policy.retry(failure)) {
synchronized (listener) {
if (cancelRecovery) {
return null;
}
while (clusterStateManager.getState() != ClusterState.ACTIVE) {
if (cancelRecovery) {
return null;
}
listener.wait();
}
}
IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(),
listener.getEntityId().getDataverse() + '.' + listener.getEntityId().getEntityName());
for (Dataset dataset : listener.getDatasets()) {
lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dataset.getDataverseName());
lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
DatasetUtil.getFullyQualifiedName(dataset));
}
synchronized (listener) {
try {
if (cancelRecovery) {
return null;
}
listener.setState(ActivityState.RECOVERING);
listener.doStart(metadataProvider);
return null;
} catch (Exception e) {
LOGGER.log(level, "Attempt to revive " + listener.getEntityId() + " failed", e);
listener.setState(ActivityState.TEMPORARILY_FAILED);
failure = e;
} finally {
metadataProvider.getLocks().reset();
}
listener.notifyAll();
}
}
// Recovery task is essntially over now either through failure or through cancellation(stop)
synchronized (listener) {
listener.notifyAll();
if (listener.getState() != ActivityState.TEMPORARILY_FAILED
// Suspend can happen at the same time, the recovery policy decides to stop... in that case, we
// must still do two things:
// 1. set the state to permanent failure.
// 2. set the entity to not running to avoid auto recovery attempt
&& listener.getState() != ActivityState.SUSPENDED) {
return null;
}
}
IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
try {
lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(),
listener.getEntityId().getDataverse() + '.' + listener.getEntityId().getEntityName());
for (Dataset dataset : listener.getDatasets()) {
MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataset.getDatasetName(),
DatasetUtil.getFullyQualifiedName(dataset));
}
synchronized (listener) {
if (cancelRecovery) {
return null;
}
if (listener.getState() == ActivityState.TEMPORARILY_FAILED) {
LOGGER.warn("Recovery for {} permanently failed", listener.getEntityId());
listener.setState(ActivityState.STOPPED);
listener.setRunning(metadataProvider, false);
}
listener.notifyAll();
}
} finally {
metadataProvider.getLocks().reset();
}
return null;
}
public Exception getFailure() {
return failure;
}
}