blob: f8abf7cd57286b8f8d54fb8d0ca241dfb12aacb1 [file] [log] [blame]
/**
* 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.hadoop.ozone.container.upgrade;
import static org.apache.hadoop.ozone.upgrade.UpgradeException.ResultCodes.PREFINALIZE_VALIDATION_FAILED;
import static org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.Status.FINALIZATION_REQUIRED;
import java.io.IOException;
import java.util.Iterator;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
import org.apache.hadoop.hdds.upgrade.HDDSLayoutVersionManager;
import org.apache.hadoop.ozone.common.Storage;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.upgrade.BasicUpgradeFinalizer;
import org.apache.hadoop.ozone.upgrade.UpgradeException;
/**
* UpgradeFinalizer for the DataNode.
*/
public class DataNodeUpgradeFinalizer extends
BasicUpgradeFinalizer<DatanodeStateMachine, HDDSLayoutVersionManager> {
public DataNodeUpgradeFinalizer(HDDSLayoutVersionManager versionManager) {
super(versionManager);
}
@Override
public void preFinalizeUpgrade(DatanodeStateMachine dsm)
throws IOException {
if(!canFinalizeDataNode(dsm)) {
// DataNode is not yet ready to finalize.
// Reset the Finalization state.
getVersionManager().setUpgradeState(FINALIZATION_REQUIRED);
String msg = "Pre Finalization checks failed on the DataNode.";
logAndEmit(msg);
throw new UpgradeException(msg, PREFINALIZE_VALIDATION_FAILED);
}
}
private boolean canFinalizeDataNode(DatanodeStateMachine dsm) {
// Lets be sure that we do not have any open container before we return
// from here. This function should be called in its own finalizer thread
// context.
Iterator<Container<?>> containerIt =
dsm.getContainer().getController().getContainers();
while (containerIt.hasNext()) {
Container ctr = containerIt.next();
ContainerProtos.ContainerDataProto.State state = ctr.getContainerState();
switch (state) {
case OPEN:
case CLOSING:
case UNHEALTHY:
LOG.warn("FinalizeUpgrade : Waiting for container to close, current "
+ "state is: {}", state);
return false;
default:
continue;
}
}
return true;
}
@Override
public void finalizeUpgrade(DatanodeStateMachine dsm)
throws UpgradeException {
super.finalizeUpgrade(dsm::getLayoutStorage);
}
@Override
public void runPrefinalizeStateActions(Storage storage,
DatanodeStateMachine dsm)
throws IOException {
super.runPrefinalizeStateActions(
lf -> ((HDDSLayoutFeature) lf)::datanodeAction, storage, dsm);
}
}