|
@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.container.common.impl;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import org.apache.commons.codec.digest.DigestUtils;
|
|
|
+import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
|
|
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
|
|
@@ -75,6 +76,8 @@ import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
|
.Result.NO_SUCH_ALGORITHM;
|
|
|
import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
|
.Result.UNABLE_TO_READ_METADATA_DB;
|
|
|
+import static org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
|
+ .Result.UNSUPPORTED_REQUEST;
|
|
|
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
|
|
|
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_META;
|
|
|
|
|
@@ -478,6 +481,95 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
|
containerMap.put(containerName, status);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public void updateContainer(Pipeline pipeline, String containerName,
|
|
|
+ ContainerData data) throws StorageContainerException{
|
|
|
+ Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
|
|
|
+ Preconditions.checkNotNull(containerName, "Container name cannot be null");
|
|
|
+ Preconditions.checkNotNull(data, "Container data cannot be null");
|
|
|
+ FileOutputStream containerStream = null;
|
|
|
+ DigestOutputStream dos = null;
|
|
|
+ MessageDigest sha = null;
|
|
|
+ File containerFileBK = null, containerFile = null;
|
|
|
+ boolean deleted = false;
|
|
|
+
|
|
|
+ if(!containerMap.containsKey(containerName)) {
|
|
|
+ throw new StorageContainerException("Container doesn't exist. Name :"
|
|
|
+ + containerName, CONTAINER_NOT_FOUND);
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
|
|
+ } catch (NoSuchAlgorithmException e) {
|
|
|
+ throw new StorageContainerException("Unable to create Message Digest,"
|
|
|
+ + " usually this is a java configuration issue.",
|
|
|
+ NO_SUCH_ALGORITHM);
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ Path location = locationManager.getContainerPath();
|
|
|
+ ContainerData orgData = containerMap.get(containerName).getContainer();
|
|
|
+ if (!orgData.isOpen()) {
|
|
|
+ throw new StorageContainerException(
|
|
|
+ "Update a closed container is not allowed. Name: " + containerName,
|
|
|
+ UNSUPPORTED_REQUEST);
|
|
|
+ }
|
|
|
+
|
|
|
+ containerFile = ContainerUtils.getContainerFile(orgData, location);
|
|
|
+ if (!containerFile.exists() || !containerFile.canWrite()) {
|
|
|
+ throw new StorageContainerException(
|
|
|
+ "Container file not exists or corrupted. Name: " + containerName,
|
|
|
+ CONTAINER_INTERNAL_ERROR);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Backup the container file
|
|
|
+ containerFileBK = File.createTempFile(
|
|
|
+ "tmp_" + System.currentTimeMillis() + "_",
|
|
|
+ containerFile.getName(), containerFile.getParentFile());
|
|
|
+ FileUtils.copyFile(containerFile, containerFileBK);
|
|
|
+
|
|
|
+ deleted = containerFile.delete();
|
|
|
+ containerStream = new FileOutputStream(containerFile);
|
|
|
+ dos = new DigestOutputStream(containerStream, sha);
|
|
|
+
|
|
|
+ ContainerProtos.ContainerData protoData = data.getProtoBufMessage();
|
|
|
+ protoData.writeDelimitedTo(dos);
|
|
|
+
|
|
|
+ // Update the in-memory map
|
|
|
+ ContainerStatus newStatus = new ContainerStatus(data, true);
|
|
|
+ containerMap.replace(containerName, newStatus);
|
|
|
+ } catch (IOException e) {
|
|
|
+ // Restore the container file from backup
|
|
|
+ if(containerFileBK != null && containerFileBK.exists() && deleted) {
|
|
|
+ if(containerFile.delete()
|
|
|
+ && containerFileBK.renameTo(containerFile)) {
|
|
|
+ throw new StorageContainerException("Container update failed,"
|
|
|
+ + " container data restored from the backup.",
|
|
|
+ CONTAINER_INTERNAL_ERROR);
|
|
|
+ } else {
|
|
|
+ throw new StorageContainerException(
|
|
|
+ "Failed to restore container data from the backup. Name: "
|
|
|
+ + containerName, CONTAINER_INTERNAL_ERROR);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ if (containerFileBK != null && containerFileBK.exists()) {
|
|
|
+ if(!containerFileBK.delete()) {
|
|
|
+ LOG.warn("Unable to delete container file backup : {}.",
|
|
|
+ containerFileBK.getAbsolutePath());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ IOUtils.closeStream(dos);
|
|
|
+ IOUtils.closeStream(containerStream);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ protected File getContainerFile(ContainerData data) throws IOException {
|
|
|
+ return ContainerUtils.getContainerFile(data,
|
|
|
+ this.locationManager.getContainerPath());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Checks if a container exists.
|
|
|
*
|