|
@@ -22,6 +22,7 @@ import com.google.common.primitives.Longs;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
|
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
|
|
@@ -37,7 +38,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
|
- .StorageContainerDatanodeProtocolProtos;
|
|
|
+ .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
|
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
|
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
|
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
|
@@ -144,7 +145,7 @@ public class SCMContainerManager implements ContainerManager {
|
|
|
.getSequentialRangeKVs(null, Integer.MAX_VALUE, null);
|
|
|
for (Map.Entry<byte[], byte[]> entry : range) {
|
|
|
ContainerInfo container = ContainerInfo.fromProtobuf(
|
|
|
- HddsProtos.SCMContainerInfo.PARSER.parseFrom(entry.getValue()));
|
|
|
+ ContainerInfoProto.PARSER.parseFrom(entry.getValue()));
|
|
|
Preconditions.checkNotNull(container);
|
|
|
containerStateManager.loadContainer(container);
|
|
|
if (container.isOpen()) {
|
|
@@ -452,7 +453,7 @@ public class SCMContainerManager implements ContainerManager {
|
|
|
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
|
|
|
}
|
|
|
ContainerInfo containerInfo = ContainerInfo.fromProtobuf(
|
|
|
- HddsProtos.SCMContainerInfo.parseFrom(containerBytes));
|
|
|
+ HddsProtos.ContainerInfoProto.parseFrom(containerBytes));
|
|
|
containerInfo.updateDeleteTransactionId(entry.getValue());
|
|
|
batch.put(dbKey, containerInfo.getProtobuf().toByteArray());
|
|
|
}
|
|
@@ -507,11 +508,11 @@ public class SCMContainerManager implements ContainerManager {
|
|
|
@Override
|
|
|
public void processContainerReports(DatanodeDetails datanodeDetails,
|
|
|
ContainerReportsProto reports) throws IOException {
|
|
|
- List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
|
|
|
+ List<ContainerReplicaProto>
|
|
|
containerInfos = reports.getReportsList();
|
|
|
PendingDeleteStatusList pendingDeleteStatusList =
|
|
|
new PendingDeleteStatusList(datanodeDetails);
|
|
|
- for (StorageContainerDatanodeProtocolProtos.ContainerInfo newInfo :
|
|
|
+ for (ContainerReplicaProto newInfo :
|
|
|
containerInfos) {
|
|
|
ContainerID id = ContainerID.valueof(newInfo.getContainerID());
|
|
|
ContainerReplica replica = ContainerReplica.newBuilder()
|
|
@@ -523,7 +524,7 @@ public class SCMContainerManager implements ContainerManager {
|
|
|
try {
|
|
|
containerStateManager.updateContainerReplica(id, replica);
|
|
|
ContainerInfo currentInfo = containerStateManager.getContainer(id);
|
|
|
- if (newInfo.getState() == LifeCycleState.CLOSED
|
|
|
+ if (newInfo.getState() == ContainerReplicaProto.State.CLOSED
|
|
|
&& currentInfo.getState() == LifeCycleState.CLOSING) {
|
|
|
currentInfo = updateContainerStateInternal(id, LifeCycleEvent.CLOSE);
|
|
|
if (!currentInfo.isOpen()) {
|
|
@@ -532,7 +533,7 @@ public class SCMContainerManager implements ContainerManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- HddsProtos.SCMContainerInfo newState =
|
|
|
+ ContainerInfoProto newState =
|
|
|
reconcileState(newInfo, currentInfo);
|
|
|
|
|
|
if (currentInfo.getDeleteTransactionId() >
|
|
@@ -567,11 +568,11 @@ public class SCMContainerManager implements ContainerManager {
|
|
|
* @param knownState - State inside SCM.
|
|
|
* @return new SCM State for this container.
|
|
|
*/
|
|
|
- private HddsProtos.SCMContainerInfo reconcileState(
|
|
|
- StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState,
|
|
|
+ private HddsProtos.ContainerInfoProto reconcileState(
|
|
|
+ ContainerReplicaProto datanodeState,
|
|
|
ContainerInfo knownState) {
|
|
|
- HddsProtos.SCMContainerInfo.Builder builder =
|
|
|
- HddsProtos.SCMContainerInfo.newBuilder();
|
|
|
+ HddsProtos.ContainerInfoProto.Builder builder =
|
|
|
+ HddsProtos.ContainerInfoProto.newBuilder();
|
|
|
builder.setContainerID(knownState.getContainerID())
|
|
|
.setPipelineID(knownState.getPipelineID().getProtobuf())
|
|
|
.setState(knownState.getState())
|