|
@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common.impl;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
-import org.apache.commons.codec.digest.DigestUtils;
|
|
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
@@ -101,7 +100,6 @@ import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
|
|
import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
|
|
import static org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result
|
|
.UNSUPPORTED_REQUEST;
|
|
.UNSUPPORTED_REQUEST;
|
|
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
|
|
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
|
|
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_META;
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* A Generic ContainerManagerImpl that will be called from Ozone
|
|
* A Generic ContainerManagerImpl that will be called from Ozone
|
|
@@ -233,18 +231,11 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
long containerID = Long.parseLong(keyName);
|
|
long containerID = Long.parseLong(keyName);
|
|
try {
|
|
try {
|
|
String containerFileName = containerName.concat(CONTAINER_EXTENSION);
|
|
String containerFileName = containerName.concat(CONTAINER_EXTENSION);
|
|
- String metaFileName = containerName.concat(CONTAINER_META);
|
|
|
|
|
|
|
|
containerStream = new FileInputStream(containerFileName);
|
|
containerStream = new FileInputStream(containerFileName);
|
|
|
|
|
|
- metaStream = new FileInputStream(metaFileName);
|
|
|
|
-
|
|
|
|
- MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
|
|
|
-
|
|
|
|
- dis = new DigestInputStream(containerStream, sha);
|
|
|
|
-
|
|
|
|
ContainerProtos.ContainerData containerDataProto =
|
|
ContainerProtos.ContainerData containerDataProto =
|
|
- ContainerProtos.ContainerData.parseDelimitedFrom(dis);
|
|
|
|
|
|
+ ContainerProtos.ContainerData.parseDelimitedFrom(containerStream);
|
|
ContainerData containerData;
|
|
ContainerData containerData;
|
|
if (containerDataProto == null) {
|
|
if (containerDataProto == null) {
|
|
// Sometimes container metadata might have been created but empty,
|
|
// Sometimes container metadata might have been created but empty,
|
|
@@ -255,19 +246,6 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
|
|
containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
|
|
- ContainerProtos.ContainerMeta meta =
|
|
|
|
- ContainerProtos.ContainerMeta.parseDelimitedFrom(metaStream);
|
|
|
|
- if (meta != null && !DigestUtils.sha256Hex(sha.digest())
|
|
|
|
- .equals(meta.getHash())) {
|
|
|
|
- // This means we were not able read data from the disk when booted the
|
|
|
|
- // datanode. We are going to rely on SCM understanding that we don't
|
|
|
|
- // have valid data for this container when we send container reports.
|
|
|
|
- // Hopefully SCM will ask us to delete this container and rebuild it.
|
|
|
|
- LOG.error("Invalid SHA found for container data. Name :{}"
|
|
|
|
- + "cowardly refusing to read invalid data", containerName);
|
|
|
|
- containerMap.put(containerID, new ContainerStatus(null));
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
|
|
|
|
ContainerStatus containerStatus = new ContainerStatus(containerData);
|
|
ContainerStatus containerStatus = new ContainerStatus(containerData);
|
|
// Initialize pending deletion blocks count in in-memory
|
|
// Initialize pending deletion blocks count in in-memory
|
|
@@ -298,7 +276,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
containerStatus.setBytesUsed(bytesUsed);
|
|
containerStatus.setBytesUsed(bytesUsed);
|
|
|
|
|
|
containerMap.put(containerID, containerStatus);
|
|
containerMap.put(containerID, containerStatus);
|
|
- } catch (IOException | NoSuchAlgorithmException ex) {
|
|
|
|
|
|
+ } catch (IOException ex) {
|
|
LOG.error("read failed for file: {} ex: {}", containerName,
|
|
LOG.error("read failed for file: {} ex: {}", containerName,
|
|
ex.getMessage());
|
|
ex.getMessage());
|
|
|
|
|
|
@@ -398,12 +376,10 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
|
|
|
|
File containerFile = ContainerUtils.getContainerFile(containerData,
|
|
File containerFile = ContainerUtils.getContainerFile(containerData,
|
|
location);
|
|
location);
|
|
- File metadataFile = ContainerUtils.getMetadataFile(containerData,
|
|
|
|
- location);
|
|
|
|
String containerName = Long.toString(containerData.getContainerID());
|
|
String containerName = Long.toString(containerData.getContainerID());
|
|
|
|
|
|
if(!overwrite) {
|
|
if(!overwrite) {
|
|
- ContainerUtils.verifyIsNewContainer(containerFile, metadataFile);
|
|
|
|
|
|
+ ContainerUtils.verifyIsNewContainer(containerFile);
|
|
metadataPath = this.locationManager.getDataPath(containerName);
|
|
metadataPath = this.locationManager.getDataPath(containerName);
|
|
metadataPath = ContainerUtils.createMetadata(metadataPath,
|
|
metadataPath = ContainerUtils.createMetadata(metadataPath,
|
|
containerName, conf);
|
|
containerName, conf);
|
|
@@ -412,7 +388,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
}
|
|
}
|
|
|
|
|
|
containerStream = new FileOutputStream(containerFile);
|
|
containerStream = new FileOutputStream(containerFile);
|
|
- metaStream = new FileOutputStream(metadataFile);
|
|
|
|
|
|
+
|
|
MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
|
MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
|
|
|
|
|
|
dos = new DigestOutputStream(containerStream, sha);
|
|
dos = new DigestOutputStream(containerStream, sha);
|
|
@@ -425,13 +401,6 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
.getProtoBufMessage();
|
|
.getProtoBufMessage();
|
|
protoData.writeDelimitedTo(dos);
|
|
protoData.writeDelimitedTo(dos);
|
|
|
|
|
|
- ContainerProtos.ContainerMeta protoMeta = ContainerProtos
|
|
|
|
- .ContainerMeta.newBuilder()
|
|
|
|
- .setFileName(containerFile.toString())
|
|
|
|
- .setHash(DigestUtils.sha256Hex(sha.digest()))
|
|
|
|
- .build();
|
|
|
|
- protoMeta.writeDelimitedTo(metaStream);
|
|
|
|
-
|
|
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
// TODO : we need to clean up partially constructed files
|
|
// TODO : we need to clean up partially constructed files
|
|
// The proper way to do would be for a thread
|
|
// The proper way to do would be for a thread
|
|
@@ -913,9 +882,6 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
.setWriteBytes(container.getWriteBytes())
|
|
.setWriteBytes(container.getWriteBytes())
|
|
.setContainerID(container.getContainer().getContainerID());
|
|
.setContainerID(container.getContainer().getContainerID());
|
|
|
|
|
|
- if (container.getContainer().getHash() != null) {
|
|
|
|
- ciBuilder.setFinalhash(container.getContainer().getHash());
|
|
|
|
- }
|
|
|
|
crBuilder.addReports(ciBuilder.build());
|
|
crBuilder.addReports(ciBuilder.build());
|
|
}
|
|
}
|
|
|
|
|