HDDS-662. Introduce ContainerReplicaState in StorageContainerManager. Contributed by Nanda kumar.
This commit is contained in:
parent
a9a63ae4a8
commit
50715c0699
|
@ -21,7 +21,7 @@ import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB
|
import org.apache.hadoop.hdds.scm.protocolPB
|
||||||
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.hdds.scm.client;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
.ContainerData;
|
.ContainerData;
|
||||||
|
|
|
@ -0,0 +1,46 @@
|
||||||
|
/**
|
||||||
|
* 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.hdds.scm.container;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Signals that ContainerException of some sort has occurred. This is parent
|
||||||
|
* of all the exceptions thrown by ContainerManager.
|
||||||
|
*/
|
||||||
|
public class ContainerException extends IOException {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code ContainerException} with {@code null}
|
||||||
|
* as its error detail message.
|
||||||
|
*/
|
||||||
|
public ContainerException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code ContainerException} with the specified detail message.
|
||||||
|
*
|
||||||
|
* @param message
|
||||||
|
* The detail message (which is saved for later retrieval
|
||||||
|
* by the {@link #getMessage()} method)
|
||||||
|
*/
|
||||||
|
public ContainerException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,6 +19,7 @@
|
||||||
package org.apache.hadoop.hdds.scm.container;
|
package org.apache.hadoop.hdds.scm.container;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.primitives.Longs;
|
||||||
import org.apache.commons.lang3.builder.CompareToBuilder;
|
import org.apache.commons.lang3.builder.CompareToBuilder;
|
||||||
import org.apache.commons.lang3.builder.EqualsBuilder;
|
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||||
|
@ -29,18 +30,17 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||||
* We are creating a specific type for this to avoid mixing this with
|
* We are creating a specific type for this to avoid mixing this with
|
||||||
* normal integers in code.
|
* normal integers in code.
|
||||||
*/
|
*/
|
||||||
public class ContainerID implements Comparable {
|
public final class ContainerID implements Comparable<ContainerID> {
|
||||||
|
|
||||||
private final long id;
|
private final long id;
|
||||||
|
|
||||||
|
// TODO: make this private.
|
||||||
/**
|
/**
|
||||||
* Constructs ContainerID.
|
* Constructs ContainerID.
|
||||||
*
|
*
|
||||||
* @param id int
|
* @param id int
|
||||||
*/
|
*/
|
||||||
public ContainerID(long id) {
|
public ContainerID(long id) {
|
||||||
Preconditions.checkState(id > 0,
|
|
||||||
"Container ID should be a positive long. "+ id);
|
|
||||||
this.id = id;
|
this.id = id;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -49,7 +49,9 @@ public class ContainerID implements Comparable {
|
||||||
* @param containerID long
|
* @param containerID long
|
||||||
* @return ContainerID.
|
* @return ContainerID.
|
||||||
*/
|
*/
|
||||||
public static ContainerID valueof(long containerID) {
|
public static ContainerID valueof(final long containerID) {
|
||||||
|
Preconditions.checkState(containerID > 0,
|
||||||
|
"Container ID should be a positive long. "+ containerID);
|
||||||
return new ContainerID(containerID);
|
return new ContainerID(containerID);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -62,8 +64,12 @@ public class ContainerID implements Comparable {
|
||||||
return id;
|
return id;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public byte[] getBytes() {
|
||||||
|
return Longs.toByteArray(id);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o) {
|
public boolean equals(final Object o) {
|
||||||
if (this == o) {
|
if (this == o) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -72,7 +78,7 @@ public class ContainerID implements Comparable {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
ContainerID that = (ContainerID) o;
|
final ContainerID that = (ContainerID) o;
|
||||||
|
|
||||||
return new EqualsBuilder()
|
return new EqualsBuilder()
|
||||||
.append(getId(), that.getId())
|
.append(getId(), that.getId())
|
||||||
|
@ -87,14 +93,8 @@ public class ContainerID implements Comparable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int compareTo(Object o) {
|
public int compareTo(final ContainerID that) {
|
||||||
Preconditions.checkNotNull(o);
|
Preconditions.checkNotNull(that);
|
||||||
if(getClass() != o.getClass()) {
|
|
||||||
throw new ClassCastException("ContainerID class expected. found:" +
|
|
||||||
o.getClass().toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
ContainerID that = (ContainerID) o;
|
|
||||||
return new CompareToBuilder()
|
return new CompareToBuilder()
|
||||||
.append(this.getId(), that.getId())
|
.append(this.getId(), that.getId())
|
||||||
.build();
|
.build();
|
||||||
|
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hdds.scm.container.common.helpers;
|
package org.apache.hadoop.hdds.scm.container;
|
||||||
|
|
||||||
import static java.lang.Math.max;
|
import static java.lang.Math.max;
|
||||||
|
|
||||||
|
@ -36,7 +36,7 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
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.ReplicationFactor;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
||||||
import org.apache.hadoop.util.Time;
|
import org.apache.hadoop.util.Time;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -62,9 +62,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
private PipelineID pipelineID;
|
private PipelineID pipelineID;
|
||||||
private ReplicationFactor replicationFactor;
|
private ReplicationFactor replicationFactor;
|
||||||
private ReplicationType replicationType;
|
private ReplicationType replicationType;
|
||||||
// Bytes allocated by SCM for clients.
|
|
||||||
private long allocatedBytes;
|
|
||||||
// Actual container usage, updated through heartbeat.
|
|
||||||
private long usedBytes;
|
private long usedBytes;
|
||||||
private long numberOfKeys;
|
private long numberOfKeys;
|
||||||
private long lastUsed;
|
private long lastUsed;
|
||||||
|
@ -84,7 +81,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
long containerID,
|
long containerID,
|
||||||
HddsProtos.LifeCycleState state,
|
HddsProtos.LifeCycleState state,
|
||||||
PipelineID pipelineID,
|
PipelineID pipelineID,
|
||||||
long allocatedBytes,
|
|
||||||
long usedBytes,
|
long usedBytes,
|
||||||
long numberOfKeys,
|
long numberOfKeys,
|
||||||
long stateEnterTime,
|
long stateEnterTime,
|
||||||
|
@ -94,7 +90,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
ReplicationType repType) {
|
ReplicationType repType) {
|
||||||
this.containerID = containerID;
|
this.containerID = containerID;
|
||||||
this.pipelineID = pipelineID;
|
this.pipelineID = pipelineID;
|
||||||
this.allocatedBytes = allocatedBytes;
|
|
||||||
this.usedBytes = usedBytes;
|
this.usedBytes = usedBytes;
|
||||||
this.numberOfKeys = numberOfKeys;
|
this.numberOfKeys = numberOfKeys;
|
||||||
this.lastUsed = Time.monotonicNow();
|
this.lastUsed = Time.monotonicNow();
|
||||||
|
@ -108,7 +103,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
|
|
||||||
public ContainerInfo(ContainerInfo info) {
|
public ContainerInfo(ContainerInfo info) {
|
||||||
this(info.getContainerID(), info.getState(), info.getPipelineID(),
|
this(info.getContainerID(), info.getState(), info.getPipelineID(),
|
||||||
info.getAllocatedBytes(), info.getUsedBytes(), info.getNumberOfKeys(),
|
info.getUsedBytes(), info.getNumberOfKeys(),
|
||||||
info.getStateEnterTime(), info.getOwner(),
|
info.getStateEnterTime(), info.getOwner(),
|
||||||
info.getDeleteTransactionId(), info.getReplicationFactor(),
|
info.getDeleteTransactionId(), info.getReplicationFactor(),
|
||||||
info.getReplicationType());
|
info.getReplicationType());
|
||||||
|
@ -123,7 +118,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
ContainerInfo.Builder builder = new ContainerInfo.Builder();
|
ContainerInfo.Builder builder = new ContainerInfo.Builder();
|
||||||
return builder.setPipelineID(
|
return builder.setPipelineID(
|
||||||
PipelineID.getFromProtobuf(info.getPipelineID()))
|
PipelineID.getFromProtobuf(info.getPipelineID()))
|
||||||
.setAllocatedBytes(info.getAllocatedBytes())
|
|
||||||
.setUsedBytes(info.getUsedBytes())
|
.setUsedBytes(info.getUsedBytes())
|
||||||
.setNumberOfKeys(info.getNumberOfKeys())
|
.setNumberOfKeys(info.getNumberOfKeys())
|
||||||
.setState(info.getState())
|
.setState(info.getState())
|
||||||
|
@ -160,20 +154,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
return pipelineID;
|
return pipelineID;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getAllocatedBytes() {
|
|
||||||
return allocatedBytes;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set Allocated bytes.
|
|
||||||
*
|
|
||||||
* @param size - newly allocated bytes -- negative size is case of deletes
|
|
||||||
* can be used.
|
|
||||||
*/
|
|
||||||
public void updateAllocatedBytes(long size) {
|
|
||||||
this.allocatedBytes += size;
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getUsedBytes() {
|
public long getUsedBytes() {
|
||||||
return usedBytes;
|
return usedBytes;
|
||||||
}
|
}
|
||||||
|
@ -211,18 +191,11 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
lastUsed = Time.monotonicNow();
|
lastUsed = Time.monotonicNow();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void allocate(long size) {
|
|
||||||
// should we also have total container size in ContainerInfo
|
|
||||||
// and check before allocating?
|
|
||||||
allocatedBytes += size;
|
|
||||||
}
|
|
||||||
|
|
||||||
public HddsProtos.SCMContainerInfo getProtobuf() {
|
public HddsProtos.SCMContainerInfo getProtobuf() {
|
||||||
HddsProtos.SCMContainerInfo.Builder builder =
|
HddsProtos.SCMContainerInfo.Builder builder =
|
||||||
HddsProtos.SCMContainerInfo.newBuilder();
|
HddsProtos.SCMContainerInfo.newBuilder();
|
||||||
Preconditions.checkState(containerID > 0);
|
Preconditions.checkState(containerID > 0);
|
||||||
return builder.setAllocatedBytes(getAllocatedBytes())
|
return builder.setContainerID(getContainerID())
|
||||||
.setContainerID(getContainerID())
|
|
||||||
.setUsedBytes(getUsedBytes())
|
.setUsedBytes(getUsedBytes())
|
||||||
.setNumberOfKeys(getNumberOfKeys()).setState(getState())
|
.setNumberOfKeys(getNumberOfKeys()).setState(getState())
|
||||||
.setStateEnterTime(getStateEnterTime()).setContainerID(getContainerID())
|
.setStateEnterTime(getStateEnterTime()).setContainerID(getContainerID())
|
||||||
|
@ -393,7 +366,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private HddsProtos.LifeCycleState state;
|
private HddsProtos.LifeCycleState state;
|
||||||
private long allocated;
|
|
||||||
private long used;
|
private long used;
|
||||||
private long keys;
|
private long keys;
|
||||||
private long stateEnterTime;
|
private long stateEnterTime;
|
||||||
|
@ -431,11 +403,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder setAllocatedBytes(long bytesAllocated) {
|
|
||||||
this.allocated = bytesAllocated;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder setUsedBytes(long bytesUsed) {
|
public Builder setUsedBytes(long bytesUsed) {
|
||||||
this.used = bytesUsed;
|
this.used = bytesUsed;
|
||||||
return this;
|
return this;
|
||||||
|
@ -462,7 +429,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
}
|
}
|
||||||
|
|
||||||
public ContainerInfo build() {
|
public ContainerInfo build() {
|
||||||
return new ContainerInfo(containerID, state, pipelineID, allocated,
|
return new ContainerInfo(containerID, state, pipelineID,
|
||||||
used, keys, stateEnterTime, owner, deleteTransactionId,
|
used, keys, stateEnterTime, owner, deleteTransactionId,
|
||||||
replicationFactor, replicationType);
|
replicationFactor, replicationType);
|
||||||
}
|
}
|
||||||
|
@ -473,7 +440,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||||
* container is either open, allocated, creating or creating.
|
* container is either open, allocated, creating or creating.
|
||||||
* Any containers in these states is managed as an open container by SCM.
|
* Any containers in these states is managed as an open container by SCM.
|
||||||
*/
|
*/
|
||||||
public boolean isContainerOpen() {
|
public boolean isOpen() {
|
||||||
return state == HddsProtos.LifeCycleState.ALLOCATED ||
|
return state == HddsProtos.LifeCycleState.ALLOCATED ||
|
||||||
state == HddsProtos.LifeCycleState.CREATING ||
|
state == HddsProtos.LifeCycleState.CREATING ||
|
||||||
state == HddsProtos.LifeCycleState.OPEN ||
|
state == HddsProtos.LifeCycleState.OPEN ||
|
|
@ -0,0 +1,44 @@
|
||||||
|
/**
|
||||||
|
* 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.hdds.scm.container;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Signals that a container is missing from ContainerManager.
|
||||||
|
*/
|
||||||
|
public class ContainerNotFoundException extends ContainerException {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code ContainerNotFoundException} with {@code null}
|
||||||
|
* as its error detail message.
|
||||||
|
*/
|
||||||
|
public ContainerNotFoundException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code ContainerNotFoundException} with the specified
|
||||||
|
* detail message.
|
||||||
|
*
|
||||||
|
* @param message
|
||||||
|
* The detail message (which is saved for later retrieval
|
||||||
|
* by the {@link #getMessage()} method)
|
||||||
|
*/
|
||||||
|
public ContainerNotFoundException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,45 @@
|
||||||
|
/**
|
||||||
|
* 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.hdds.scm.container;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Signals that a ContainerReplica is missing from the Container in
|
||||||
|
* ContainerManager.
|
||||||
|
*/
|
||||||
|
public class ContainerReplicaNotFoundException extends ContainerException {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code ContainerReplicaNotFoundException} with {@code null}
|
||||||
|
* as its error detail message.
|
||||||
|
*/
|
||||||
|
public ContainerReplicaNotFoundException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code ContainerReplicaNotFoundException} with the
|
||||||
|
* specified detail message.
|
||||||
|
*
|
||||||
|
* @param message
|
||||||
|
* The detail message (which is saved for later retrieval
|
||||||
|
* by the {@link #getMessage()} method)
|
||||||
|
*/
|
||||||
|
public ContainerReplicaNotFoundException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
}
|
|
@ -22,6 +22,7 @@ import java.util.Comparator;
|
||||||
import org.apache.commons.lang3.builder.EqualsBuilder;
|
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class wraps ozone container info.
|
* Class wraps ozone container info.
|
||||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.hdds.scm.protocol;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolPro
|
||||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InChillModeResponseProto;
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InChillModeResponseProto;
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
|
|
@ -33,7 +33,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolPro
|
||||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto;
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto;
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
|
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
|
|
@ -135,16 +135,13 @@ message SCMContainerInfo {
|
||||||
required int64 containerID = 1;
|
required int64 containerID = 1;
|
||||||
required LifeCycleState state = 2;
|
required LifeCycleState state = 2;
|
||||||
optional PipelineID pipelineID = 3;
|
optional PipelineID pipelineID = 3;
|
||||||
// This is not total size of container, but space allocated by SCM for
|
required uint64 usedBytes = 4;
|
||||||
// clients to write blocks
|
required uint64 numberOfKeys = 5;
|
||||||
required uint64 allocatedBytes = 4;
|
optional int64 stateEnterTime = 6;
|
||||||
required uint64 usedBytes = 5;
|
required string owner = 7;
|
||||||
required uint64 numberOfKeys = 6;
|
optional int64 deleteTransactionId = 8;
|
||||||
optional int64 stateEnterTime = 7;
|
required ReplicationFactor replicationFactor = 9;
|
||||||
required string owner = 8;
|
required ReplicationType replicationType = 10;
|
||||||
optional int64 deleteTransactionId = 9;
|
|
||||||
required ReplicationFactor replicationFactor = 10;
|
|
||||||
required ReplicationType replicationType = 11;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
message ContainerWithPipeline {
|
message ContainerWithPipeline {
|
||||||
|
|
|
@ -75,10 +75,10 @@ public class CommandStatusReportPublisher extends
|
||||||
|
|
||||||
iterator.forEachRemaining(key -> {
|
iterator.forEachRemaining(key -> {
|
||||||
CommandStatus cmdStatus = map.get(key);
|
CommandStatus cmdStatus = map.get(key);
|
||||||
builder.addCmdStatus(cmdStatus.getProtoBufMessage());
|
|
||||||
// If status is still pending then don't remove it from map as
|
// If status is still pending then don't remove it from map as
|
||||||
// CommandHandler will change its status when it works on this command.
|
// CommandHandler will change its status when it works on this command.
|
||||||
if (!cmdStatus.getStatus().equals(Status.PENDING)) {
|
if (!cmdStatus.getStatus().equals(Status.PENDING)) {
|
||||||
|
builder.addCmdStatus(cmdStatus.getProtoBufMessage());
|
||||||
map.remove(key);
|
map.remove(key);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
|
@ -153,19 +153,10 @@ public class TestReportPublisher {
|
||||||
.build();
|
.build();
|
||||||
cmdStatusMap.put(obj1.getCmdId(), obj1);
|
cmdStatusMap.put(obj1.getCmdId(), obj1);
|
||||||
cmdStatusMap.put(obj2.getCmdId(), obj2);
|
cmdStatusMap.put(obj2.getCmdId(), obj2);
|
||||||
Assert.assertEquals("Should publish report with 2 status objects", 2,
|
// We are not sending the commands whose status is PENDING.
|
||||||
|
Assert.assertEquals("Should publish report with 2 status objects", 1,
|
||||||
((CommandStatusReportPublisher) publisher).getReport()
|
((CommandStatusReportPublisher) publisher).getReport()
|
||||||
.getCmdStatusCount());
|
.getCmdStatusCount());
|
||||||
Assert.assertEquals(
|
|
||||||
"Next report should have 1 status objects as command status o"
|
|
||||||
+ "bjects are still in Pending state",
|
|
||||||
1, ((CommandStatusReportPublisher) publisher).getReport()
|
|
||||||
.getCmdStatusCount());
|
|
||||||
Assert.assertTrue(
|
|
||||||
"Next report should have 1 status objects as command status "
|
|
||||||
+ "objects are still in Pending state",
|
|
||||||
((CommandStatusReportPublisher) publisher).getReport()
|
|
||||||
.getCmdStatusList().get(0).getStatus().equals(Status.PENDING));
|
|
||||||
executorService.shutdown();
|
executorService.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.hadoop.hdds.scm.ScmUtils;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
@ -69,7 +69,6 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
// Currently only user of the block service is Ozone, CBlock manages blocks
|
// Currently only user of the block service is Ozone, CBlock manages blocks
|
||||||
// by itself and does not rely on the Block service offered by SCM.
|
// by itself and does not rely on the Block service offered by SCM.
|
||||||
|
|
||||||
private final NodeManager nodeManager;
|
|
||||||
private final ContainerManager containerManager;
|
private final ContainerManager containerManager;
|
||||||
|
|
||||||
private final long containerSize;
|
private final long containerSize;
|
||||||
|
@ -95,7 +94,6 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
final NodeManager nodeManager, final ContainerManager containerManager,
|
final NodeManager nodeManager, final ContainerManager containerManager,
|
||||||
EventPublisher eventPublisher)
|
EventPublisher eventPublisher)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
this.nodeManager = nodeManager;
|
|
||||||
this.containerManager = containerManager;
|
this.containerManager = containerManager;
|
||||||
|
|
||||||
this.containerSize = (long)conf.getStorageSize(
|
this.containerSize = (long)conf.getStorageSize(
|
||||||
|
@ -226,8 +224,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
// USER of the containers. So there might be cases where a different
|
// USER of the containers. So there might be cases where a different
|
||||||
// USER has few containers in ALLOCATED state, which will result in
|
// USER has few containers in ALLOCATED state, which will result in
|
||||||
// false positive.
|
// false positive.
|
||||||
if (!containerManager.getStateManager().getContainerStateMap()
|
if (!containerManager.getContainers(HddsProtos.LifeCycleState.ALLOCATED)
|
||||||
.getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
|
||||||
.isEmpty()) {
|
.isEmpty()) {
|
||||||
// Since the above check can result in false positive, we have to do
|
// Since the above check can result in false positive, we have to do
|
||||||
// the actual check and find out if there are containers in ALLOCATED
|
// the actual check and find out if there are containers in ALLOCATED
|
||||||
|
@ -242,7 +239,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
HddsProtos.LifeCycleState.ALLOCATED);
|
HddsProtos.LifeCycleState.ALLOCATED);
|
||||||
if (containerWithPipeline != null) {
|
if (containerWithPipeline != null) {
|
||||||
containerManager.updateContainerState(
|
containerManager.updateContainerState(
|
||||||
containerWithPipeline.getContainerInfo().getContainerID(),
|
containerWithPipeline.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
return newBlock(containerWithPipeline,
|
return newBlock(containerWithPipeline,
|
||||||
HddsProtos.LifeCycleState.ALLOCATED);
|
HddsProtos.LifeCycleState.ALLOCATED);
|
||||||
|
@ -268,8 +265,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
// state, we have to check again as we only hold a read lock.
|
// state, we have to check again as we only hold a read lock.
|
||||||
// Some other thread might have pre-allocated container in meantime.
|
// Some other thread might have pre-allocated container in meantime.
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
if (!containerManager.getStateManager().getContainerStateMap()
|
if (!containerManager.getContainers(HddsProtos.LifeCycleState.ALLOCATED)
|
||||||
.getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
|
||||||
.isEmpty()) {
|
.isEmpty()) {
|
||||||
containerWithPipeline = containerManager
|
containerWithPipeline = containerManager
|
||||||
.getMatchingContainerWithPipeline(size, owner, type, factor,
|
.getMatchingContainerWithPipeline(size, owner, type, factor,
|
||||||
|
@ -285,7 +281,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
|
|
||||||
if (containerWithPipeline != null) {
|
if (containerWithPipeline != null) {
|
||||||
containerManager.updateContainerState(
|
containerManager.updateContainerState(
|
||||||
containerWithPipeline.getContainerInfo().getContainerID(),
|
containerWithPipeline.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
return newBlock(containerWithPipeline,
|
return newBlock(containerWithPipeline,
|
||||||
HddsProtos.LifeCycleState.ALLOCATED);
|
HddsProtos.LifeCycleState.ALLOCATED);
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.hadoop.hdds.scm.block;
|
package org.apache.hadoop.hdds.scm.block;
|
||||||
|
|
||||||
import com.google.common.collect.ArrayListMultimap;
|
import com.google.common.collect.ArrayListMultimap;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
|
@ -60,8 +61,9 @@ public class DatanodeDeletedBlockTransactions {
|
||||||
Pipeline pipeline = null;
|
Pipeline pipeline = null;
|
||||||
try {
|
try {
|
||||||
ContainerWithPipeline containerWithPipeline =
|
ContainerWithPipeline containerWithPipeline =
|
||||||
containerManager.getContainerWithPipeline(tx.getContainerID());
|
containerManager.getContainerWithPipeline(
|
||||||
if (containerWithPipeline.getContainerInfo().isContainerOpen()
|
ContainerID.valueof(tx.getContainerID()));
|
||||||
|
if (containerWithPipeline.getContainerInfo().isOpen()
|
||||||
|| containerWithPipeline.getPipeline().isEmpty()) {
|
|| containerWithPipeline.getPipeline().isEmpty()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.hdds.protocol.proto
|
||||||
.DeleteBlockTransactionResult;
|
.DeleteBlockTransactionResult;
|
||||||
import org.apache.hadoop.hdds.scm.command
|
import org.apache.hadoop.hdds.scm.command
|
||||||
.CommandStatusReportHandler.DeleteBlockStatus;
|
.CommandStatusReportHandler.DeleteBlockStatus;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
|
@ -258,8 +259,8 @@ public class DeletedBlockLogImpl
|
||||||
|
|
||||||
dnsWithCommittedTxn.add(dnID);
|
dnsWithCommittedTxn.add(dnID);
|
||||||
Pipeline pipeline =
|
Pipeline pipeline =
|
||||||
containerManager.getContainerWithPipeline(containerId)
|
containerManager.getContainerWithPipeline(
|
||||||
.getPipeline();
|
ContainerID.valueof(containerId)).getPipeline();
|
||||||
Collection<DatanodeDetails> containerDnsDetails =
|
Collection<DatanodeDetails> containerDnsDetails =
|
||||||
pipeline.getDatanodes().values();
|
pipeline.getDatanodes().values();
|
||||||
// The delete entry can be safely removed from the log if all the
|
// The delete entry can be safely removed from the log if all the
|
||||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.hadoop.hdds.scm.container;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
|
@ -61,7 +60,7 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
|
||||||
ContainerInfo info;
|
ContainerInfo info;
|
||||||
try {
|
try {
|
||||||
containerWithPipeline =
|
containerWithPipeline =
|
||||||
containerManager.getContainerWithPipeline(containerID.getId());
|
containerManager.getContainerWithPipeline(containerID);
|
||||||
info = containerWithPipeline.getContainerInfo();
|
info = containerWithPipeline.getContainerInfo();
|
||||||
if (info == null) {
|
if (info == null) {
|
||||||
LOG.error("Failed to update the container state. Container with id : {}"
|
LOG.error("Failed to update the container state. Container with id : {}"
|
||||||
|
@ -81,8 +80,8 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
|
||||||
// We cannot close a container in ALLOCATED state, moving the
|
// We cannot close a container in ALLOCATED state, moving the
|
||||||
// container to CREATING state, this should eventually
|
// container to CREATING state, this should eventually
|
||||||
// timeout and the container will be moved to DELETING state.
|
// timeout and the container will be moved to DELETING state.
|
||||||
LOG.debug("Closing container {} in {} state", containerID, state);
|
LOG.debug("Closing container #{} in {} state", containerID, state);
|
||||||
containerManager.updateContainerState(containerID.getId(),
|
containerManager.updateContainerState(containerID,
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
break;
|
break;
|
||||||
case CREATING:
|
case CREATING:
|
||||||
|
@ -91,7 +90,7 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
|
||||||
LOG.debug("Closing container {} in {} state", containerID, state);
|
LOG.debug("Closing container {} in {} state", containerID, state);
|
||||||
break;
|
break;
|
||||||
case OPEN:
|
case OPEN:
|
||||||
containerManager.updateContainerState(containerID.getId(),
|
containerManager.updateContainerState(containerID,
|
||||||
HddsProtos.LifeCycleEvent.FINALIZE);
|
HddsProtos.LifeCycleEvent.FINALIZE);
|
||||||
fireCloseContainerEvents(containerWithPipeline, info, publisher);
|
fireCloseContainerEvents(containerWithPipeline, info, publisher);
|
||||||
break;
|
break;
|
||||||
|
@ -101,16 +100,15 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
|
||||||
case CLOSED:
|
case CLOSED:
|
||||||
case DELETING:
|
case DELETING:
|
||||||
case DELETED:
|
case DELETED:
|
||||||
LOG.info(
|
LOG.info("Cannot close container #{}, it is already in {} state.",
|
||||||
"container with id : {} is in {} state and need not be closed.",
|
containerID.getId(), state);
|
||||||
containerID.getId(), info.getState());
|
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new IOException(
|
throw new IOException("Invalid container state for container #"
|
||||||
"Invalid container state for container " + containerID);
|
+ containerID);
|
||||||
}
|
}
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
LOG.error("Failed to update the container state for" + "container : {}"
|
LOG.error("Failed to update the container state for container #{}"
|
||||||
+ containerID, ex);
|
+ containerID, ex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -125,13 +123,14 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
|
||||||
info.getReplicationType(), info.getPipelineID());
|
info.getReplicationType(), info.getPipelineID());
|
||||||
|
|
||||||
Pipeline pipeline = containerWithPipeline.getPipeline();
|
Pipeline pipeline = containerWithPipeline.getPipeline();
|
||||||
pipeline.getMachines().stream().map(
|
pipeline.getMachines().stream()
|
||||||
datanode -> new CommandForDatanode<>(datanode.getUuid(),
|
.map(node ->
|
||||||
closeContainerCommand)).forEach((command) -> {
|
new CommandForDatanode<>(node.getUuid(), closeContainerCommand))
|
||||||
publisher.fireEvent(DATANODE_COMMAND, command);
|
.forEach(command -> publisher.fireEvent(DATANODE_COMMAND, command));
|
||||||
});
|
|
||||||
publisher.fireEvent(CLOSE_CONTAINER_RETRYABLE_REQ,
|
publisher.fireEvent(CLOSE_CONTAINER_RETRYABLE_REQ,
|
||||||
new CloseContainerRetryableReq(containerID));
|
new CloseContainerRetryableReq(containerID));
|
||||||
|
|
||||||
LOG.trace("Issuing {} on Pipeline {} for container", closeContainerCommand,
|
LOG.trace("Issuing {} on Pipeline {} for container", closeContainerCommand,
|
||||||
pipeline, containerID);
|
pipeline, containerID);
|
||||||
}
|
}
|
||||||
|
|
|
@ -88,7 +88,8 @@ public class CloseContainerWatcher extends
|
||||||
publisher) {
|
publisher) {
|
||||||
try {
|
try {
|
||||||
// Check if container is still open
|
// Check if container is still open
|
||||||
if (containerManager.getContainer(containerID).isContainerOpen()) {
|
if (containerManager.getContainer(
|
||||||
|
ContainerID.valueof(containerID)).isOpen()) {
|
||||||
publisher.fireEvent(SCMEvents.CLOSE_CONTAINER,
|
publisher.fireEvent(SCMEvents.CLOSE_CONTAINER,
|
||||||
ContainerID.valueof(containerID));
|
ContainerID.valueof(containerID));
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
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.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
||||||
|
@ -31,13 +30,31 @@ import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
// TODO: Write extensive java doc.
|
||||||
|
// This is the main interface of ContainerManager.
|
||||||
/**
|
/**
|
||||||
* ContainerManager class contains the mapping from a name to a pipeline
|
* ContainerManager class contains the mapping from a name to a pipeline
|
||||||
* mapping. This is used by SCM when allocating new locations and when
|
* mapping. This is used by SCM when allocating new locations and when
|
||||||
* looking up a key.
|
* looking up a key.
|
||||||
*/
|
*/
|
||||||
public interface ContainerManager extends Closeable {
|
public interface ContainerManager extends Closeable {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns all the containers managed by ContainerManager.
|
||||||
|
*
|
||||||
|
* @return List of ContainerInfo
|
||||||
|
*/
|
||||||
|
List<ContainerInfo> getContainers();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns all the containers which are in the specified state.
|
||||||
|
*
|
||||||
|
* @return List of ContainerInfo
|
||||||
|
*/
|
||||||
|
List<ContainerInfo> getContainers(HddsProtos.LifeCycleState state);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the ContainerInfo from the container ID.
|
* Returns the ContainerInfo from the container ID.
|
||||||
*
|
*
|
||||||
|
@ -45,7 +62,8 @@ public interface ContainerManager extends Closeable {
|
||||||
* @return - ContainerInfo such as creation state and the pipeline.
|
* @return - ContainerInfo such as creation state and the pipeline.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
ContainerInfo getContainer(long containerID) throws IOException;
|
ContainerInfo getContainer(ContainerID containerID)
|
||||||
|
throws ContainerNotFoundException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the ContainerInfo from the container ID.
|
* Returns the ContainerInfo from the container ID.
|
||||||
|
@ -54,8 +72,8 @@ public interface ContainerManager extends Closeable {
|
||||||
* @return - ContainerWithPipeline such as creation state and the pipeline.
|
* @return - ContainerWithPipeline such as creation state and the pipeline.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
ContainerWithPipeline getContainerWithPipeline(long containerID)
|
ContainerWithPipeline getContainerWithPipeline(ContainerID containerID)
|
||||||
throws IOException;
|
throws ContainerNotFoundException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns containers under certain conditions.
|
* Returns containers under certain conditions.
|
||||||
|
@ -72,8 +90,7 @@ public interface ContainerManager extends Closeable {
|
||||||
* @return a list of container.
|
* @return a list of container.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
List<ContainerInfo> listContainer(long startContainerID, int count)
|
List<ContainerInfo> listContainer(ContainerID startContainerID, int count);
|
||||||
throws IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Allocates a new container for a given keyName and replication factor.
|
* Allocates a new container for a given keyName and replication factor.
|
||||||
|
@ -93,7 +110,7 @@ public interface ContainerManager extends Closeable {
|
||||||
* @param containerID - Container ID
|
* @param containerID - Container ID
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void deleteContainer(long containerID) throws IOException;
|
void deleteContainer(ContainerID containerID) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Update container state.
|
* Update container state.
|
||||||
|
@ -102,23 +119,44 @@ public interface ContainerManager extends Closeable {
|
||||||
* @return - new container state
|
* @return - new container state
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
HddsProtos.LifeCycleState updateContainerState(long containerID,
|
HddsProtos.LifeCycleState updateContainerState(ContainerID containerID,
|
||||||
HddsProtos.LifeCycleEvent event) throws IOException;
|
HddsProtos.LifeCycleEvent event) throws IOException;
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the container State Manager.
|
|
||||||
* @return ContainerStateManager
|
|
||||||
*/
|
|
||||||
ContainerStateManager getStateManager();
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Process container report from Datanode.
|
* Process container report from Datanode.
|
||||||
*
|
*
|
||||||
* @param reports Container report
|
* @param reports Container report
|
||||||
*/
|
*/
|
||||||
void processContainerReports(DatanodeDetails datanodeDetails,
|
void processContainerReports(DatanodeDetails datanodeDetails,
|
||||||
ContainerReportsProto reports, boolean isRegisterCall)
|
ContainerReportsProto reports) throws IOException;
|
||||||
throws IOException;
|
|
||||||
|
/**
|
||||||
|
* Returns the latest list of replicas for given containerId.
|
||||||
|
*
|
||||||
|
* @param containerID Container ID
|
||||||
|
* @return Set of ContainerReplica
|
||||||
|
*/
|
||||||
|
Set<ContainerReplica> getContainerReplicas(ContainerID containerID)
|
||||||
|
throws ContainerNotFoundException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adds a container Replica for the given Container.
|
||||||
|
*
|
||||||
|
* @param containerID Container ID
|
||||||
|
* @param replica ContainerReplica
|
||||||
|
*/
|
||||||
|
void updateContainerReplica(ContainerID containerID, ContainerReplica replica)
|
||||||
|
throws ContainerNotFoundException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove a container Replica form a given Container.
|
||||||
|
*
|
||||||
|
* @param containerID Container ID
|
||||||
|
* @param replica ContainerReplica
|
||||||
|
* @return True of dataNode is removed successfully else false.
|
||||||
|
*/
|
||||||
|
void removeContainerReplica(ContainerID containerID, ContainerReplica replica)
|
||||||
|
throws ContainerNotFoundException, ContainerReplicaNotFoundException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Update deleteTransactionId according to deleteTransactionMap.
|
* Update deleteTransactionId according to deleteTransactionMap.
|
||||||
|
|
|
@ -0,0 +1,197 @@
|
||||||
|
/**
|
||||||
|
* 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.hdds.scm.container;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.commons.lang3.builder.CompareToBuilder;
|
||||||
|
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||||
|
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* In-memory state of a container replica.
|
||||||
|
*/
|
||||||
|
public final class ContainerReplica implements Comparable<ContainerReplica> {
|
||||||
|
|
||||||
|
final private ContainerID containerID;
|
||||||
|
final private DatanodeDetails datanodeDetails;
|
||||||
|
final private UUID placeOfBirth;
|
||||||
|
|
||||||
|
private Long sequenceId;
|
||||||
|
|
||||||
|
|
||||||
|
private ContainerReplica(ContainerID containerID, DatanodeDetails datanode,
|
||||||
|
UUID originNodeId) {
|
||||||
|
this.containerID = containerID;
|
||||||
|
this.datanodeDetails = datanode;
|
||||||
|
this.placeOfBirth = originNodeId;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setSequenceId(Long seqId) {
|
||||||
|
sequenceId = seqId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the DatanodeDetails to which this replica belongs.
|
||||||
|
*
|
||||||
|
* @return DatanodeDetails
|
||||||
|
*/
|
||||||
|
public DatanodeDetails getDatanodeDetails() {
|
||||||
|
return datanodeDetails;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the UUID of Datanode where this replica originated.
|
||||||
|
*
|
||||||
|
* @return UUID
|
||||||
|
*/
|
||||||
|
public UUID getOriginDatanodeId() {
|
||||||
|
return placeOfBirth;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the Sequence Id of this replica.
|
||||||
|
*
|
||||||
|
* @return Sequence Id
|
||||||
|
*/
|
||||||
|
public Long getSequenceId() {
|
||||||
|
return sequenceId;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return new HashCodeBuilder(61, 71)
|
||||||
|
.append(containerID)
|
||||||
|
.append(datanodeDetails)
|
||||||
|
.toHashCode();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final ContainerReplica that = (ContainerReplica) o;
|
||||||
|
|
||||||
|
return new EqualsBuilder()
|
||||||
|
.append(containerID, that.containerID)
|
||||||
|
.append(datanodeDetails, that.datanodeDetails)
|
||||||
|
.isEquals();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compareTo(ContainerReplica that) {
|
||||||
|
Preconditions.checkNotNull(that);
|
||||||
|
return new CompareToBuilder()
|
||||||
|
.append(this.containerID, that.containerID)
|
||||||
|
.append(this.datanodeDetails, that.datanodeDetails)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a new Builder to construct ContainerReplica.
|
||||||
|
*
|
||||||
|
* @return ContainerReplicaBuilder
|
||||||
|
*/
|
||||||
|
public static ContainerReplicaBuilder newBuilder() {
|
||||||
|
return new ContainerReplicaBuilder();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Used for building ContainerReplica instance.
|
||||||
|
*/
|
||||||
|
public static class ContainerReplicaBuilder {
|
||||||
|
|
||||||
|
private ContainerID containerID;
|
||||||
|
private DatanodeDetails datanode;
|
||||||
|
private UUID placeOfBirth;
|
||||||
|
private Long sequenceId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set Container Id.
|
||||||
|
*
|
||||||
|
* @param containerId ContainerID
|
||||||
|
* @return ContainerReplicaBuilder
|
||||||
|
*/
|
||||||
|
public ContainerReplicaBuilder setContainerID(
|
||||||
|
final ContainerID containerId) {
|
||||||
|
containerID = containerId;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set DatanodeDetails.
|
||||||
|
*
|
||||||
|
* @param datanodeDetails DatanodeDetails
|
||||||
|
* @return ContainerReplicaBuilder
|
||||||
|
*/
|
||||||
|
public ContainerReplicaBuilder setDatanodeDetails(
|
||||||
|
DatanodeDetails datanodeDetails) {
|
||||||
|
datanode = datanodeDetails;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set replica origin node id.
|
||||||
|
*
|
||||||
|
* @param originNodeId origin node UUID
|
||||||
|
* @return ContainerReplicaBuilder
|
||||||
|
*/
|
||||||
|
public ContainerReplicaBuilder setOriginNodeId(UUID originNodeId) {
|
||||||
|
placeOfBirth = originNodeId;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set sequence Id of the replica.
|
||||||
|
*
|
||||||
|
* @param seqId container sequence Id
|
||||||
|
* @return ContainerReplicaBuilder
|
||||||
|
*/
|
||||||
|
public ContainerReplicaBuilder setSequenceId(long seqId) {
|
||||||
|
sequenceId = seqId;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs new ContainerReplicaBuilder.
|
||||||
|
*
|
||||||
|
* @return ContainerReplicaBuilder
|
||||||
|
*/
|
||||||
|
public ContainerReplica build() {
|
||||||
|
Preconditions.checkNotNull(containerID,
|
||||||
|
"Container Id can't be null");
|
||||||
|
Preconditions.checkNotNull(datanode,
|
||||||
|
"DatanodeDetails can't be null");
|
||||||
|
ContainerReplica replica = new ContainerReplica(containerID, datanode,
|
||||||
|
Optional.ofNullable(placeOfBirth).orElse(datanode.getUuid()));
|
||||||
|
Optional.ofNullable(sequenceId).ifPresent(replica::setSequenceId);
|
||||||
|
return replica;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -24,11 +24,9 @@ import java.util.stream.Collectors;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.replication.ReplicationActivityStatus;
|
import org.apache.hadoop.hdds.scm.container.replication.ReplicationActivityStatus;
|
||||||
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
||||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||||
import org.apache.hadoop.hdds.scm.node.states.ReportResult;
|
import org.apache.hadoop.hdds.scm.node.states.ReportResult;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode;
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode;
|
||||||
|
@ -49,11 +47,7 @@ public class ContainerReportHandler implements
|
||||||
LoggerFactory.getLogger(ContainerReportHandler.class);
|
LoggerFactory.getLogger(ContainerReportHandler.class);
|
||||||
|
|
||||||
private final NodeManager nodeManager;
|
private final NodeManager nodeManager;
|
||||||
|
|
||||||
private final ContainerManager containerManager;
|
private final ContainerManager containerManager;
|
||||||
|
|
||||||
private ContainerStateManager containerStateManager;
|
|
||||||
|
|
||||||
private ReplicationActivityStatus replicationStatus;
|
private ReplicationActivityStatus replicationStatus;
|
||||||
|
|
||||||
public ContainerReportHandler(ContainerManager containerManager,
|
public ContainerReportHandler(ContainerManager containerManager,
|
||||||
|
@ -62,7 +56,6 @@ public class ContainerReportHandler implements
|
||||||
Preconditions.checkNotNull(containerManager);
|
Preconditions.checkNotNull(containerManager);
|
||||||
Preconditions.checkNotNull(nodeManager);
|
Preconditions.checkNotNull(nodeManager);
|
||||||
Preconditions.checkNotNull(replicationActivityStatus);
|
Preconditions.checkNotNull(replicationActivityStatus);
|
||||||
this.containerStateManager = containerManager.getStateManager();
|
|
||||||
this.nodeManager = nodeManager;
|
this.nodeManager = nodeManager;
|
||||||
this.containerManager = containerManager;
|
this.containerManager = containerManager;
|
||||||
this.replicationStatus = replicationActivityStatus;
|
this.replicationStatus = replicationActivityStatus;
|
||||||
|
@ -81,7 +74,7 @@ public class ContainerReportHandler implements
|
||||||
|
|
||||||
//update state in container db and trigger close container events
|
//update state in container db and trigger close container events
|
||||||
containerManager
|
containerManager
|
||||||
.processContainerReports(datanodeOrigin, containerReport, false);
|
.processContainerReports(datanodeOrigin, containerReport);
|
||||||
|
|
||||||
Set<ContainerID> containerIds = containerReport.getReportsList().stream()
|
Set<ContainerID> containerIds = containerReport.getReportsList().stream()
|
||||||
.map(StorageContainerDatanodeProtocolProtos
|
.map(StorageContainerDatanodeProtocolProtos
|
||||||
|
@ -97,13 +90,21 @@ public class ContainerReportHandler implements
|
||||||
.setContainersForDatanode(datanodeOrigin.getUuid(), containerIds);
|
.setContainersForDatanode(datanodeOrigin.getUuid(), containerIds);
|
||||||
|
|
||||||
for (ContainerID containerID : reportResult.getMissingEntries()) {
|
for (ContainerID containerID : reportResult.getMissingEntries()) {
|
||||||
containerStateManager
|
final ContainerReplica replica = ContainerReplica.newBuilder()
|
||||||
.removeContainerReplica(containerID, datanodeOrigin);
|
.setContainerID(containerID)
|
||||||
|
.setDatanodeDetails(datanodeOrigin)
|
||||||
|
.build();
|
||||||
|
containerManager
|
||||||
|
.removeContainerReplica(containerID, replica);
|
||||||
checkReplicationState(containerID, publisher);
|
checkReplicationState(containerID, publisher);
|
||||||
}
|
}
|
||||||
|
|
||||||
for (ContainerID containerID : reportResult.getNewEntries()) {
|
for (ContainerID containerID : reportResult.getNewEntries()) {
|
||||||
containerStateManager.addContainerReplica(containerID, datanodeOrigin);
|
final ContainerReplica replica = ContainerReplica.newBuilder()
|
||||||
|
.setContainerID(containerID)
|
||||||
|
.setDatanodeDetails(datanodeOrigin)
|
||||||
|
.build();
|
||||||
|
containerManager.updateContainerReplica(containerID, replica);
|
||||||
checkReplicationState(containerID, publisher);
|
checkReplicationState(containerID, publisher);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -116,35 +117,30 @@ public class ContainerReportHandler implements
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkReplicationState(ContainerID containerID,
|
private void checkReplicationState(ContainerID containerID,
|
||||||
EventPublisher publisher)
|
EventPublisher publisher) {
|
||||||
throws SCMException {
|
try {
|
||||||
ContainerInfo container = containerStateManager.getContainer(containerID);
|
ContainerInfo container = containerManager.getContainer(containerID);
|
||||||
|
replicateIfNeeded(container, publisher);
|
||||||
if (container == null) {
|
} catch (ContainerNotFoundException ex) {
|
||||||
//warning unknown container
|
|
||||||
LOG.warn(
|
LOG.warn(
|
||||||
"Container is missing from containerStateManager. Can't request "
|
"Container is missing from containerStateManager. Can't request "
|
||||||
+ "replication. {}",
|
+ "replication. {}",
|
||||||
containerID);
|
containerID);
|
||||||
return;
|
|
||||||
}
|
|
||||||
if (container.isContainerOpen()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
ReplicationRequest replicationState =
|
|
||||||
containerStateManager.checkReplicationState(containerID);
|
|
||||||
if (replicationState != null) {
|
|
||||||
if (replicationStatus.isReplicationEnabled()) {
|
|
||||||
publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
|
|
||||||
replicationState);
|
|
||||||
} else {
|
|
||||||
LOG.warn(
|
|
||||||
"Over/under replicated container but the replication is not "
|
|
||||||
+ "(yet) enabled: "
|
|
||||||
+ replicationState.toString());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void replicateIfNeeded(ContainerInfo container,
|
||||||
|
EventPublisher publisher) throws ContainerNotFoundException {
|
||||||
|
if (!container.isOpen() && replicationStatus.isReplicationEnabled()) {
|
||||||
|
final int existingReplicas = containerManager
|
||||||
|
.getContainerReplicas(container.containerID()).size();
|
||||||
|
final int expectedReplicas = container.getReplicationFactor().getNumber();
|
||||||
|
if (existingReplicas != expectedReplicas) {
|
||||||
|
publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
|
||||||
|
new ReplicationRequest(container.getContainerID(),
|
||||||
|
existingReplicas, expectedReplicas));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,17 +17,12 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hdds.scm.container;
|
package org.apache.hadoop.hdds.scm.container;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.conf.StorageUnit;
|
import org.apache.hadoop.conf.StorageUnit;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.states.ContainerState;
|
import org.apache.hadoop.hdds.scm.container.states.ContainerState;
|
||||||
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
|
@ -45,11 +40,8 @@ import org.apache.hadoop.util.Time;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.Closeable;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.NavigableSet;
|
import java.util.NavigableSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -116,7 +108,7 @@ import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
||||||
* TimeOut Delete Container State Machine - if the container creating times out,
|
* TimeOut Delete Container State Machine - if the container creating times out,
|
||||||
* then Container State manager decides to delete the container.
|
* then Container State manager decides to delete the container.
|
||||||
*/
|
*/
|
||||||
public class ContainerStateManager implements Closeable {
|
public class ContainerStateManager {
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(ContainerStateManager.class);
|
LoggerFactory.getLogger(ContainerStateManager.class);
|
||||||
|
|
||||||
|
@ -135,11 +127,10 @@ public class ContainerStateManager implements Closeable {
|
||||||
* TODO : Add Container Tags so we know which containers are owned by SCM.
|
* TODO : Add Container Tags so we know which containers are owned by SCM.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public ContainerStateManager(Configuration configuration,
|
public ContainerStateManager(final Configuration configuration) {
|
||||||
ContainerManager containerManager, PipelineSelector pipelineSelector) {
|
|
||||||
|
|
||||||
// Initialize the container state machine.
|
// Initialize the container state machine.
|
||||||
Set<HddsProtos.LifeCycleState> finalStates = new HashSet();
|
final Set<HddsProtos.LifeCycleState> finalStates = new HashSet();
|
||||||
|
|
||||||
// These are the steady states of a container.
|
// These are the steady states of a container.
|
||||||
finalStates.add(LifeCycleState.OPEN);
|
finalStates.add(LifeCycleState.OPEN);
|
||||||
|
@ -155,22 +146,9 @@ public class ContainerStateManager implements Closeable {
|
||||||
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
|
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
|
||||||
StorageUnit.BYTES);
|
StorageUnit.BYTES);
|
||||||
|
|
||||||
lastUsedMap = new ConcurrentHashMap<>();
|
this.lastUsedMap = new ConcurrentHashMap<>();
|
||||||
containerCount = new AtomicLong(0);
|
this.containerCount = new AtomicLong(0);
|
||||||
containers = new ContainerStateMap();
|
this.containers = new ContainerStateMap();
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Return the info of all the containers kept by the in-memory mapping.
|
|
||||||
*
|
|
||||||
* @return the list of all container info.
|
|
||||||
*/
|
|
||||||
public List<ContainerInfo> getAllContainers() {
|
|
||||||
List<ContainerInfo> list = new ArrayList<>();
|
|
||||||
|
|
||||||
//No Locking needed since the return value is an immutable map.
|
|
||||||
containers.getContainerMap().forEach((key, value) -> list.add(value));
|
|
||||||
return list;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -244,17 +222,15 @@ public class ContainerStateManager implements Closeable {
|
||||||
LifeCycleEvent.CLEANUP);
|
LifeCycleEvent.CLEANUP);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void addExistingContainer(ContainerInfo containerInfo)
|
void loadContainer(final ContainerInfo containerInfo)
|
||||||
throws SCMException {
|
throws SCMException {
|
||||||
containers.addContainer(containerInfo);
|
containers.addContainer(containerInfo);
|
||||||
long containerID = containerInfo.getContainerID();
|
containerCount.set(Long.max(
|
||||||
if (containerCount.get() < containerID) {
|
containerInfo.getContainerID(), containerCount.get()));
|
||||||
containerCount.set(containerID);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* allocates a new container based on the type, replication etc.
|
* Allocates a new container based on the type, replication etc.
|
||||||
*
|
*
|
||||||
* @param selector -- Pipeline selector class.
|
* @param selector -- Pipeline selector class.
|
||||||
* @param type -- Replication type.
|
* @param type -- Replication type.
|
||||||
|
@ -262,25 +238,22 @@ public class ContainerStateManager implements Closeable {
|
||||||
* @return ContainerWithPipeline
|
* @return ContainerWithPipeline
|
||||||
* @throws IOException on Failure.
|
* @throws IOException on Failure.
|
||||||
*/
|
*/
|
||||||
public ContainerWithPipeline allocateContainer(PipelineSelector selector,
|
ContainerInfo allocateContainer(final PipelineSelector selector,
|
||||||
HddsProtos.ReplicationType type,
|
final HddsProtos.ReplicationType type,
|
||||||
HddsProtos.ReplicationFactor replicationFactor, String owner)
|
final HddsProtos.ReplicationFactor replicationFactor, final String owner)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
Pipeline pipeline = selector.getReplicationPipeline(type,
|
final Pipeline pipeline = selector.getReplicationPipeline(type,
|
||||||
replicationFactor);
|
replicationFactor);
|
||||||
|
|
||||||
Preconditions.checkNotNull(pipeline, "Pipeline type=%s/"
|
Preconditions.checkNotNull(pipeline, "Pipeline type=%s/"
|
||||||
+ "replication=%s couldn't be found for the new container. "
|
+ "replication=%s couldn't be found for the new container. "
|
||||||
+ "Do you have enough nodes?", type, replicationFactor);
|
+ "Do you have enough nodes?", type, replicationFactor);
|
||||||
|
|
||||||
long containerID = containerCount.incrementAndGet();
|
final long containerID = containerCount.incrementAndGet();
|
||||||
ContainerInfo containerInfo = new ContainerInfo.Builder()
|
final ContainerInfo containerInfo = new ContainerInfo.Builder()
|
||||||
.setState(HddsProtos.LifeCycleState.ALLOCATED)
|
.setState(HddsProtos.LifeCycleState.ALLOCATED)
|
||||||
.setPipelineID(pipeline.getId())
|
.setPipelineID(pipeline.getId())
|
||||||
// This is bytes allocated for blocks inside container, not the
|
|
||||||
// container size
|
|
||||||
.setAllocatedBytes(0)
|
|
||||||
.setUsedBytes(0)
|
.setUsedBytes(0)
|
||||||
.setNumberOfKeys(0)
|
.setNumberOfKeys(0)
|
||||||
.setStateEnterTime(Time.monotonicNow())
|
.setStateEnterTime(Time.monotonicNow())
|
||||||
|
@ -294,35 +267,34 @@ public class ContainerStateManager implements Closeable {
|
||||||
Preconditions.checkNotNull(containerInfo);
|
Preconditions.checkNotNull(containerInfo);
|
||||||
containers.addContainer(containerInfo);
|
containers.addContainer(containerInfo);
|
||||||
LOG.trace("New container allocated: {}", containerInfo);
|
LOG.trace("New container allocated: {}", containerInfo);
|
||||||
return new ContainerWithPipeline(containerInfo, pipeline);
|
return containerInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Update the Container State to the next state.
|
* Update the Container State to the next state.
|
||||||
*
|
*
|
||||||
* @param info - ContainerInfo
|
* @param containerID - ContainerID
|
||||||
* @param event - LifeCycle Event
|
* @param event - LifeCycle Event
|
||||||
* @return Updated ContainerInfo.
|
* @return Updated ContainerInfo.
|
||||||
* @throws SCMException on Failure.
|
* @throws SCMException on Failure.
|
||||||
*/
|
*/
|
||||||
public ContainerInfo updateContainerState(ContainerInfo
|
ContainerInfo updateContainerState(final ContainerID containerID,
|
||||||
info, HddsProtos.LifeCycleEvent event) throws SCMException {
|
final HddsProtos.LifeCycleEvent event)
|
||||||
LifeCycleState newState;
|
throws SCMException, ContainerNotFoundException {
|
||||||
|
final ContainerInfo info = containers.getContainerInfo(containerID);
|
||||||
try {
|
try {
|
||||||
newState = this.stateMachine.getNextState(info.getState(), event);
|
final LifeCycleState newState = stateMachine.getNextState(
|
||||||
|
info.getState(), event);
|
||||||
|
containers.updateState(containerID, info.getState(), newState);
|
||||||
|
return containers.getContainerInfo(containerID);
|
||||||
} catch (InvalidStateTransitionException ex) {
|
} catch (InvalidStateTransitionException ex) {
|
||||||
String error = String.format("Failed to update container state %s, " +
|
String error = String.format("Failed to update container state %s, " +
|
||||||
"reason: invalid state transition from state: %s upon " +
|
"reason: invalid state transition from state: %s upon " +
|
||||||
"event: %s.",
|
"event: %s.",
|
||||||
info.getContainerID(), info.getState(), event);
|
containerID, info.getState(), event);
|
||||||
LOG.error(error);
|
LOG.error(error);
|
||||||
throw new SCMException(error, FAILED_TO_CHANGE_CONTAINER_STATE);
|
throw new SCMException(error, FAILED_TO_CHANGE_CONTAINER_STATE);
|
||||||
}
|
}
|
||||||
|
|
||||||
// This is a post condition after executing getNextState.
|
|
||||||
Preconditions.checkNotNull(newState);
|
|
||||||
containers.updateState(info, info.getState(), newState);
|
|
||||||
return containers.getContainerInfo(info);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -331,10 +303,10 @@ public class ContainerStateManager implements Closeable {
|
||||||
* @return ContainerInfo
|
* @return ContainerInfo
|
||||||
* @throws SCMException - on Error.
|
* @throws SCMException - on Error.
|
||||||
*/
|
*/
|
||||||
public ContainerInfo updateContainerInfo(ContainerInfo info)
|
ContainerInfo updateContainerInfo(final ContainerInfo info)
|
||||||
throws SCMException {
|
throws ContainerNotFoundException {
|
||||||
containers.updateContainerInfo(info);
|
containers.updateContainerInfo(info);
|
||||||
return containers.getContainerInfo(info);
|
return containers.getContainerInfo(info.containerID());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -343,11 +315,16 @@ public class ContainerStateManager implements Closeable {
|
||||||
* @param deleteTransactionMap maps containerId to its new
|
* @param deleteTransactionMap maps containerId to its new
|
||||||
* deleteTransactionID
|
* deleteTransactionID
|
||||||
*/
|
*/
|
||||||
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap) {
|
void updateDeleteTransactionId(
|
||||||
for (Map.Entry<Long, Long> entry : deleteTransactionMap.entrySet()) {
|
final Map<Long, Long> deleteTransactionMap) {
|
||||||
containers.getContainerMap().get(ContainerID.valueof(entry.getKey()))
|
deleteTransactionMap.forEach((k, v) -> {
|
||||||
.updateDeleteTransactionId(entry.getValue());
|
try {
|
||||||
}
|
containers.getContainerInfo(ContainerID.valueof(k))
|
||||||
|
.updateDeleteTransactionId(v);
|
||||||
|
} catch (ContainerNotFoundException e) {
|
||||||
|
LOG.warn("Exception while updating delete transaction id.", e);
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -360,12 +337,12 @@ public class ContainerStateManager implements Closeable {
|
||||||
* @param state - State of the Container-- {Open, Allocated etc.}
|
* @param state - State of the Container-- {Open, Allocated etc.}
|
||||||
* @return ContainerInfo, null if there is no match found.
|
* @return ContainerInfo, null if there is no match found.
|
||||||
*/
|
*/
|
||||||
public ContainerInfo getMatchingContainer(final long size,
|
ContainerInfo getMatchingContainer(final long size,
|
||||||
String owner, ReplicationType type, ReplicationFactor factor,
|
String owner, ReplicationType type, ReplicationFactor factor,
|
||||||
LifeCycleState state) {
|
LifeCycleState state) {
|
||||||
|
|
||||||
// Find containers that match the query spec, if no match return null.
|
// Find containers that match the query spec, if no match return null.
|
||||||
NavigableSet<ContainerID> matchingSet =
|
final NavigableSet<ContainerID> matchingSet =
|
||||||
containers.getMatchingContainerIDs(state, owner, factor, type);
|
containers.getMatchingContainerIDs(state, owner, factor, type);
|
||||||
if (matchingSet == null || matchingSet.size() == 0) {
|
if (matchingSet == null || matchingSet.size() == 0) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -373,11 +350,9 @@ public class ContainerStateManager implements Closeable {
|
||||||
|
|
||||||
// Get the last used container and find container above the last used
|
// Get the last used container and find container above the last used
|
||||||
// container ID.
|
// container ID.
|
||||||
ContainerState key = new ContainerState(owner, type, factor);
|
final ContainerState key = new ContainerState(owner, type, factor);
|
||||||
ContainerID lastID = lastUsedMap.get(key);
|
final ContainerID lastID = lastUsedMap
|
||||||
if (lastID == null) {
|
.getOrDefault(key, matchingSet.first());
|
||||||
lastID = matchingSet.first();
|
|
||||||
}
|
|
||||||
|
|
||||||
// There is a small issue here. The first time, we will skip the first
|
// There is a small issue here. The first time, we will skip the first
|
||||||
// container. But in most cases it will not matter.
|
// container. But in most cases it will not matter.
|
||||||
|
@ -401,32 +376,47 @@ public class ContainerStateManager implements Closeable {
|
||||||
resultSet = matchingSet.headSet(lastID, true);
|
resultSet = matchingSet.headSet(lastID, true);
|
||||||
selectedContainer = findContainerWithSpace(size, resultSet, owner);
|
selectedContainer = findContainerWithSpace(size, resultSet, owner);
|
||||||
}
|
}
|
||||||
// Update the allocated Bytes on this container.
|
|
||||||
if (selectedContainer != null) {
|
|
||||||
selectedContainer.updateAllocatedBytes(size);
|
|
||||||
}
|
|
||||||
return selectedContainer;
|
return selectedContainer;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerInfo findContainerWithSpace(long size,
|
private ContainerInfo findContainerWithSpace(final long size,
|
||||||
NavigableSet<ContainerID> searchSet, String owner) {
|
final NavigableSet<ContainerID> searchSet, final String owner) {
|
||||||
// Get the container with space to meet our request.
|
try {
|
||||||
for (ContainerID id : searchSet) {
|
// Get the container with space to meet our request.
|
||||||
ContainerInfo containerInfo = containers.getContainerInfo(id);
|
for (ContainerID id : searchSet) {
|
||||||
if (containerInfo.getAllocatedBytes() + size <= this.containerSize) {
|
final ContainerInfo containerInfo = containers.getContainerInfo(id);
|
||||||
containerInfo.updateLastUsedTime();
|
if (containerInfo.getUsedBytes() + size <= this.containerSize) {
|
||||||
|
containerInfo.updateLastUsedTime();
|
||||||
|
|
||||||
ContainerState key = new ContainerState(owner,
|
final ContainerState key = new ContainerState(owner,
|
||||||
containerInfo.getReplicationType(),
|
containerInfo.getReplicationType(),
|
||||||
containerInfo.getReplicationFactor());
|
containerInfo.getReplicationFactor());
|
||||||
lastUsedMap.put(key, containerInfo.containerID());
|
lastUsedMap.put(key, containerInfo.containerID());
|
||||||
return containerInfo;
|
return containerInfo;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
} catch (ContainerNotFoundException e) {
|
||||||
|
// This should not happen!
|
||||||
|
LOG.warn("Exception while finding container with space", e);
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Set<ContainerID> getAllContainerIDs() {
|
||||||
|
return containers.getAllContainerIDs();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns Containers by State.
|
||||||
|
*
|
||||||
|
* @param state - State - Open, Closed etc.
|
||||||
|
* @return List of containers by state.
|
||||||
|
*/
|
||||||
|
Set<ContainerID> getContainerIDsByState(final LifeCycleState state) {
|
||||||
|
return containers.getContainerIDsByState(state);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a set of ContainerIDs that match the Container.
|
* Returns a set of ContainerIDs that match the Container.
|
||||||
*
|
*
|
||||||
|
@ -436,39 +426,25 @@ public class ContainerStateManager implements Closeable {
|
||||||
* @param state - Current State, like Open, Close etc.
|
* @param state - Current State, like Open, Close etc.
|
||||||
* @return Set of containers that match the specific query parameters.
|
* @return Set of containers that match the specific query parameters.
|
||||||
*/
|
*/
|
||||||
public NavigableSet<ContainerID> getMatchingContainerIDs(
|
NavigableSet<ContainerID> getMatchingContainerIDs(final String owner,
|
||||||
String owner, ReplicationType type, ReplicationFactor factor,
|
final ReplicationType type, final ReplicationFactor factor,
|
||||||
LifeCycleState state) {
|
final LifeCycleState state) {
|
||||||
return containers.getMatchingContainerIDs(state, owner,
|
return containers.getMatchingContainerIDs(state, owner,
|
||||||
factor, type);
|
factor, type);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the containerInfo with pipeline for the given container id.
|
|
||||||
* @param selector -- Pipeline selector class.
|
|
||||||
* @param containerID id of the container
|
|
||||||
* @return ContainerInfo containerInfo
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public ContainerWithPipeline getContainer(PipelineSelector selector,
|
|
||||||
ContainerID containerID) {
|
|
||||||
ContainerInfo info = containers.getContainerInfo(containerID.getId());
|
|
||||||
Pipeline pipeline = selector.getPipeline(info.getPipelineID());
|
|
||||||
return new ContainerWithPipeline(info, pipeline);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the containerInfo for the given container id.
|
* Returns the containerInfo for the given container id.
|
||||||
* @param containerID id of the container
|
* @param containerID id of the container
|
||||||
* @return ContainerInfo containerInfo
|
* @return ContainerInfo containerInfo
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public ContainerInfo getContainer(ContainerID containerID) {
|
ContainerInfo getContainer(final ContainerID containerID)
|
||||||
|
throws ContainerNotFoundException {
|
||||||
return containers.getContainerInfo(containerID);
|
return containers.getContainerInfo(containerID);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
void close() throws IOException {
|
||||||
public void close() throws IOException {
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -478,8 +454,8 @@ public class ContainerStateManager implements Closeable {
|
||||||
* @param containerID
|
* @param containerID
|
||||||
* @return Set<DatanodeDetails>
|
* @return Set<DatanodeDetails>
|
||||||
*/
|
*/
|
||||||
public Set<DatanodeDetails> getContainerReplicas(ContainerID containerID)
|
Set<ContainerReplica> getContainerReplicas(
|
||||||
throws SCMException {
|
final ContainerID containerID) throws ContainerNotFoundException {
|
||||||
return containers.getContainerReplicas(containerID);
|
return containers.getContainerReplicas(containerID);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -487,53 +463,29 @@ public class ContainerStateManager implements Closeable {
|
||||||
* Add a container Replica for given DataNode.
|
* Add a container Replica for given DataNode.
|
||||||
*
|
*
|
||||||
* @param containerID
|
* @param containerID
|
||||||
* @param dn
|
* @param replica
|
||||||
*/
|
*/
|
||||||
public void addContainerReplica(ContainerID containerID, DatanodeDetails dn) {
|
void updateContainerReplica(final ContainerID containerID,
|
||||||
containers.addContainerReplica(containerID, dn);
|
final ContainerReplica replica) throws ContainerNotFoundException {
|
||||||
|
containers.updateContainerReplica(containerID, replica);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Remove a container Replica for given DataNode.
|
* Remove a container Replica for given DataNode.
|
||||||
*
|
*
|
||||||
* @param containerID
|
* @param containerID
|
||||||
* @param dn
|
* @param replica
|
||||||
* @return True of dataNode is removed successfully else false.
|
* @return True of dataNode is removed successfully else false.
|
||||||
*/
|
*/
|
||||||
public boolean removeContainerReplica(ContainerID containerID,
|
void removeContainerReplica(final ContainerID containerID,
|
||||||
DatanodeDetails dn) throws SCMException {
|
final ContainerReplica replica)
|
||||||
return containers.removeContainerReplica(containerID, dn);
|
throws ContainerNotFoundException, ContainerReplicaNotFoundException {
|
||||||
|
containers.removeContainerReplica(containerID, replica);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
void removeContainer(final ContainerID containerID)
|
||||||
* Compare the existing replication number with the expected one.
|
throws ContainerNotFoundException {
|
||||||
*/
|
containers.removeContainer(containerID);
|
||||||
public ReplicationRequest checkReplicationState(ContainerID containerID)
|
|
||||||
throws SCMException {
|
|
||||||
int existingReplicas = getContainerReplicas(containerID).size();
|
|
||||||
int expectedReplicas = getContainer(containerID)
|
|
||||||
.getReplicationFactor().getNumber();
|
|
||||||
if (existingReplicas != expectedReplicas) {
|
|
||||||
return new ReplicationRequest(containerID.getId(), existingReplicas,
|
|
||||||
expectedReplicas);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Checks if the container is open.
|
|
||||||
*/
|
|
||||||
public boolean isOpen(ContainerID containerID) {
|
|
||||||
Preconditions.checkNotNull(containerID);
|
|
||||||
ContainerInfo container = Preconditions
|
|
||||||
.checkNotNull(getContainer(containerID),
|
|
||||||
"Container can't be found " + containerID);
|
|
||||||
return container.isContainerOpen();
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
public ContainerStateMap getContainerStateMap() {
|
|
||||||
return containers;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,12 +24,10 @@ import org.apache.hadoop.conf.StorageUnit;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
|
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
|
@ -55,19 +53,23 @@ import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.charset.Charset;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.locks.Lock;
|
import java.util.concurrent.locks.Lock;
|
||||||
import java.util.concurrent.locks.ReentrantLock;
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||||
.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
|
.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||||
.OZONE_SCM_CONTAINER_SIZE;
|
.OZONE_SCM_CONTAINER_SIZE;
|
||||||
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_DEFAULT;
|
||||||
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_MB;
|
||||||
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
||||||
.FAILED_TO_CHANGE_CONTAINER_STATE;
|
.FAILED_TO_CHANGE_CONTAINER_STATE;
|
||||||
import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
|
import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
|
||||||
|
@ -82,10 +84,7 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(SCMContainerManager
|
private static final Logger LOG = LoggerFactory.getLogger(SCMContainerManager
|
||||||
.class);
|
.class);
|
||||||
|
|
||||||
private final NodeManager nodeManager;
|
|
||||||
private final long cacheSize;
|
|
||||||
private final Lock lock;
|
private final Lock lock;
|
||||||
private final Charset encoding = Charset.forName("UTF-8");
|
|
||||||
private final MetadataStore containerStore;
|
private final MetadataStore containerStore;
|
||||||
private final PipelineSelector pipelineSelector;
|
private final PipelineSelector pipelineSelector;
|
||||||
private final ContainerStateManager containerStateManager;
|
private final ContainerStateManager containerStateManager;
|
||||||
|
@ -100,83 +99,97 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
* @param nodeManager - NodeManager so that we can get the nodes that are
|
* @param nodeManager - NodeManager so that we can get the nodes that are
|
||||||
* healthy to place new
|
* healthy to place new
|
||||||
* containers.
|
* containers.
|
||||||
* @param cacheSizeMB - Amount of memory reserved for the LSM tree to cache
|
|
||||||
* its nodes. This is
|
|
||||||
* passed to LevelDB and this memory is allocated in Native code space.
|
* passed to LevelDB and this memory is allocated in Native code space.
|
||||||
* CacheSize is specified
|
* CacheSize is specified
|
||||||
* in MB.
|
* in MB.
|
||||||
* @throws IOException on Failure.
|
* @throws IOException on Failure.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public SCMContainerManager(
|
public SCMContainerManager(final Configuration conf,
|
||||||
final Configuration conf, final NodeManager nodeManager, final int
|
final NodeManager nodeManager, final EventPublisher eventPublisher)
|
||||||
cacheSizeMB, EventPublisher eventPublisher) throws IOException {
|
throws IOException {
|
||||||
this.nodeManager = nodeManager;
|
|
||||||
this.cacheSize = cacheSizeMB;
|
|
||||||
|
|
||||||
File metaDir = getOzoneMetaDirPath(conf);
|
final File metaDir = getOzoneMetaDirPath(conf);
|
||||||
|
final File containerDBPath = new File(metaDir, SCM_CONTAINER_DB);
|
||||||
|
final int cacheSize = conf.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
|
||||||
|
OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
|
||||||
|
|
||||||
// Write the container name to pipeline mapping.
|
this.containerStore = MetadataStoreBuilder.newBuilder()
|
||||||
File containerDBPath = new File(metaDir, SCM_CONTAINER_DB);
|
.setConf(conf)
|
||||||
containerStore =
|
.setDbFile(containerDBPath)
|
||||||
MetadataStoreBuilder.newBuilder()
|
.setCacheSize(cacheSize * OzoneConsts.MB)
|
||||||
.setConf(conf)
|
.build();
|
||||||
.setDbFile(containerDBPath)
|
|
||||||
.setCacheSize(this.cacheSize * OzoneConsts.MB)
|
|
||||||
.build();
|
|
||||||
|
|
||||||
this.lock = new ReentrantLock();
|
this.lock = new ReentrantLock();
|
||||||
|
this.size = (long) conf.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
|
||||||
size = (long)conf.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
|
|
||||||
OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
|
OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
|
||||||
|
|
||||||
this.pipelineSelector = new PipelineSelector(nodeManager,
|
this.pipelineSelector = new PipelineSelector(nodeManager,
|
||||||
conf, eventPublisher, cacheSizeMB);
|
conf, eventPublisher, cacheSize);
|
||||||
|
this.containerStateManager = new ContainerStateManager(conf);
|
||||||
this.containerStateManager =
|
|
||||||
new ContainerStateManager(conf, this, pipelineSelector);
|
|
||||||
LOG.trace("Container State Manager created.");
|
|
||||||
|
|
||||||
this.eventPublisher = eventPublisher;
|
this.eventPublisher = eventPublisher;
|
||||||
|
|
||||||
long containerCreationLeaseTimeout = conf.getTimeDuration(
|
final long containerCreationLeaseTimeout = conf.getTimeDuration(
|
||||||
ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT,
|
ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT,
|
||||||
ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT_DEFAULT,
|
ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT_DEFAULT,
|
||||||
TimeUnit.MILLISECONDS);
|
TimeUnit.MILLISECONDS);
|
||||||
containerLeaseManager = new LeaseManager<>("ContainerCreation",
|
this.containerLeaseManager = new LeaseManager<>("ContainerCreation",
|
||||||
containerCreationLeaseTimeout);
|
containerCreationLeaseTimeout);
|
||||||
containerLeaseManager.start();
|
this.containerLeaseManager.start();
|
||||||
|
|
||||||
loadExistingContainers();
|
loadExistingContainers();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void loadExistingContainers() {
|
private void loadExistingContainers() throws IOException {
|
||||||
|
List<Map.Entry<byte[], byte[]>> range = containerStore
|
||||||
List<ContainerInfo> containerList;
|
.getSequentialRangeKVs(null, Integer.MAX_VALUE, null);
|
||||||
try {
|
for (Map.Entry<byte[], byte[]> entry : range) {
|
||||||
containerList = listContainer(0, Integer.MAX_VALUE);
|
ContainerInfo container = ContainerInfo.fromProtobuf(
|
||||||
|
HddsProtos.SCMContainerInfo.PARSER.parseFrom(entry.getValue()));
|
||||||
// if there are no container to load, let us return.
|
Preconditions.checkNotNull(container);
|
||||||
if (containerList == null || containerList.size() == 0) {
|
containerStateManager.loadContainer(container);
|
||||||
LOG.info("No containers to load for this cluster.");
|
pipelineSelector.addContainerToPipeline(
|
||||||
return;
|
container.getPipelineID(), container.getContainerID());
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
if (!e.getMessage().equals("No container exists in current db")) {
|
|
||||||
LOG.error("Could not list the containers", e);
|
|
||||||
}
|
|
||||||
return;
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
// TODO: remove this later.
|
||||||
|
public ContainerStateManager getContainerStateManager() {
|
||||||
|
return containerStateManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<ContainerInfo> getContainers() {
|
||||||
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
for (ContainerInfo container : containerList) {
|
return containerStateManager.getAllContainerIDs().stream().map(id -> {
|
||||||
containerStateManager.addExistingContainer(container);
|
try {
|
||||||
pipelineSelector.addContainerToPipeline(
|
return containerStateManager.getContainer(id);
|
||||||
container.getPipelineID(), container.getContainerID());
|
} catch (ContainerNotFoundException e) {
|
||||||
}
|
// How can this happen?
|
||||||
} catch (SCMException ex) {
|
return null;
|
||||||
LOG.error("Unable to create a container information. ", ex);
|
}
|
||||||
// Fix me, what is the proper shutdown procedure for SCM ??
|
}).filter(Objects::nonNull).collect(Collectors.toList());
|
||||||
// System.exit(1) // Should we exit here?
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<ContainerInfo> getContainers(LifeCycleState state) {
|
||||||
|
lock.lock();
|
||||||
|
try {
|
||||||
|
return containerStateManager.getContainerIDsByState(state).stream()
|
||||||
|
.map(id -> {
|
||||||
|
try {
|
||||||
|
return containerStateManager.getContainer(id);
|
||||||
|
} catch (ContainerNotFoundException e) {
|
||||||
|
// How can this happen?
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}).filter(Objects::nonNull).collect(Collectors.toList());
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -184,26 +197,9 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public ContainerInfo getContainer(final long containerID) throws
|
public ContainerInfo getContainer(final ContainerID containerID)
|
||||||
IOException {
|
throws ContainerNotFoundException {
|
||||||
ContainerInfo containerInfo;
|
return containerStateManager.getContainer(containerID);
|
||||||
lock.lock();
|
|
||||||
try {
|
|
||||||
byte[] containerBytes = containerStore.get(
|
|
||||||
Longs.toByteArray(containerID));
|
|
||||||
if (containerBytes == null) {
|
|
||||||
throw new SCMException(
|
|
||||||
"Specified key does not exist. key : " + containerID,
|
|
||||||
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
|
|
||||||
}
|
|
||||||
|
|
||||||
HddsProtos.SCMContainerInfo temp = HddsProtos.SCMContainerInfo.PARSER
|
|
||||||
.parseFrom(containerBytes);
|
|
||||||
containerInfo = ContainerInfo.fromProtobuf(temp);
|
|
||||||
return containerInfo;
|
|
||||||
} finally {
|
|
||||||
lock.unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -217,38 +213,29 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public ContainerWithPipeline getContainerWithPipeline(long containerID)
|
public ContainerWithPipeline getContainerWithPipeline(ContainerID containerID)
|
||||||
throws IOException {
|
throws ContainerNotFoundException {
|
||||||
ContainerInfo contInfo;
|
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
byte[] containerBytes = containerStore.get(
|
final ContainerInfo contInfo = getContainer(containerID);
|
||||||
Longs.toByteArray(containerID));
|
|
||||||
if (containerBytes == null) {
|
|
||||||
throw new SCMException(
|
|
||||||
"Specified key does not exist. key : " + containerID,
|
|
||||||
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
|
|
||||||
}
|
|
||||||
HddsProtos.SCMContainerInfo temp = HddsProtos.SCMContainerInfo.PARSER
|
|
||||||
.parseFrom(containerBytes);
|
|
||||||
contInfo = ContainerInfo.fromProtobuf(temp);
|
|
||||||
|
|
||||||
Pipeline pipeline;
|
Pipeline pipeline;
|
||||||
String leaderId = "";
|
String leaderId = "";
|
||||||
if (contInfo.isContainerOpen()) {
|
if (contInfo.isOpen()) {
|
||||||
// If pipeline with given pipeline Id already exist return it
|
// If pipeline with given pipeline Id already exist return it
|
||||||
pipeline = pipelineSelector.getPipeline(contInfo.getPipelineID());
|
pipeline = pipelineSelector.getPipeline(contInfo.getPipelineID());
|
||||||
} else {
|
} else {
|
||||||
// For close containers create pipeline from datanodes with replicas
|
// For close containers create pipeline from datanodes with replicas
|
||||||
Set<DatanodeDetails> dnWithReplicas = containerStateManager
|
Set<ContainerReplica> dnWithReplicas = containerStateManager
|
||||||
.getContainerReplicas(contInfo.containerID());
|
.getContainerReplicas(contInfo.containerID());
|
||||||
if (!dnWithReplicas.isEmpty()) {
|
if (!dnWithReplicas.isEmpty()) {
|
||||||
leaderId = dnWithReplicas.iterator().next().getUuidString();
|
leaderId = dnWithReplicas.iterator().next()
|
||||||
|
.getDatanodeDetails().getUuidString();
|
||||||
}
|
}
|
||||||
pipeline = new Pipeline(leaderId, contInfo.getState(),
|
pipeline = new Pipeline(leaderId, contInfo.getState(),
|
||||||
ReplicationType.STAND_ALONE, contInfo.getReplicationFactor(),
|
ReplicationType.STAND_ALONE, contInfo.getReplicationFactor(),
|
||||||
PipelineID.randomId());
|
PipelineID.randomId());
|
||||||
dnWithReplicas.forEach(pipeline::addMember);
|
dnWithReplicas.stream().map(ContainerReplica::getDatanodeDetails).
|
||||||
|
forEach(pipeline::addMember);
|
||||||
}
|
}
|
||||||
return new ContainerWithPipeline(contInfo, pipeline);
|
return new ContainerWithPipeline(contInfo, pipeline);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -260,33 +247,32 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<ContainerInfo> listContainer(long startContainerID,
|
public List<ContainerInfo> listContainer(ContainerID startContainerID,
|
||||||
int count) throws IOException {
|
int count) {
|
||||||
List<ContainerInfo> containerList = new ArrayList<>();
|
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
if (containerStore.isEmpty()) {
|
final long startId = startContainerID == null ?
|
||||||
throw new IOException("No container exists in current db");
|
0 : startContainerID.getId();
|
||||||
}
|
final List<ContainerID> containersIds =
|
||||||
byte[] startKey = startContainerID <= 0 ? null :
|
new ArrayList<>(containerStateManager.getAllContainerIDs());
|
||||||
Longs.toByteArray(startContainerID);
|
Collections.sort(containersIds);
|
||||||
List<Map.Entry<byte[], byte[]>> range =
|
|
||||||
containerStore.getSequentialRangeKVs(startKey, count, null);
|
|
||||||
|
|
||||||
// Transform the values into the pipelines.
|
return containersIds.stream()
|
||||||
// TODO: filter by container state
|
.filter(id -> id.getId() > startId)
|
||||||
for (Map.Entry<byte[], byte[]> entry : range) {
|
.limit(count)
|
||||||
ContainerInfo containerInfo =
|
.map(id -> {
|
||||||
ContainerInfo.fromProtobuf(
|
try {
|
||||||
HddsProtos.SCMContainerInfo.PARSER.parseFrom(
|
return containerStateManager.getContainer(id);
|
||||||
entry.getValue()));
|
} catch (ContainerNotFoundException ex) {
|
||||||
Preconditions.checkNotNull(containerInfo);
|
// This can never happen, as we hold lock no one else can remove
|
||||||
containerList.add(containerInfo);
|
// the container after we got the container ids.
|
||||||
}
|
LOG.warn("Container Missing.", ex);
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}).collect(Collectors.toList());
|
||||||
} finally {
|
} finally {
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
}
|
}
|
||||||
return containerList;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -298,29 +284,35 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
* @throws IOException - Exception
|
* @throws IOException - Exception
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public ContainerWithPipeline allocateContainer(
|
public ContainerWithPipeline allocateContainer(final ReplicationType type,
|
||||||
ReplicationType type,
|
final ReplicationFactor replicationFactor, final String owner)
|
||||||
ReplicationFactor replicationFactor,
|
|
||||||
String owner)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
ContainerInfo containerInfo;
|
|
||||||
ContainerWithPipeline containerWithPipeline;
|
|
||||||
|
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
containerWithPipeline = containerStateManager.allocateContainer(
|
final ContainerInfo containerInfo; containerInfo = containerStateManager
|
||||||
pipelineSelector, type, replicationFactor, owner);
|
.allocateContainer(pipelineSelector, type, replicationFactor, owner);
|
||||||
containerInfo = containerWithPipeline.getContainerInfo();
|
final Pipeline pipeline = pipelineSelector.getPipeline(
|
||||||
|
containerInfo.getPipelineID());
|
||||||
|
|
||||||
byte[] containerIDBytes = Longs.toByteArray(
|
try {
|
||||||
containerInfo.getContainerID());
|
final byte[] containerIDBytes = Longs.toByteArray(
|
||||||
containerStore.put(containerIDBytes, containerInfo.getProtobuf()
|
containerInfo.getContainerID());
|
||||||
.toByteArray());
|
containerStore.put(containerIDBytes,
|
||||||
|
containerInfo.getProtobuf().toByteArray());
|
||||||
|
} catch (IOException ex) {
|
||||||
|
// If adding to containerStore fails, we should remove the container
|
||||||
|
// from in-memory map.
|
||||||
|
try {
|
||||||
|
containerStateManager.removeContainer(containerInfo.containerID());
|
||||||
|
} catch (ContainerNotFoundException cnfe) {
|
||||||
|
// No need to worry much, everything is going as planned.
|
||||||
|
}
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
return new ContainerWithPipeline(containerInfo, pipeline);
|
||||||
} finally {
|
} finally {
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
}
|
}
|
||||||
return containerWithPipeline;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -332,18 +324,24 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
* specified key.
|
* specified key.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void deleteContainer(long containerID) throws IOException {
|
public void deleteContainer(ContainerID containerID) throws IOException {
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
byte[] dbKey = Longs.toByteArray(containerID);
|
containerStateManager.removeContainer(containerID);
|
||||||
byte[] containerBytes = containerStore.get(dbKey);
|
final byte[] dbKey = Longs.toByteArray(containerID.getId());
|
||||||
if (containerBytes == null) {
|
final byte[] containerBytes = containerStore.get(dbKey);
|
||||||
throw new SCMException(
|
if (containerBytes != null) {
|
||||||
"Failed to delete container " + containerID + ", reason : " +
|
containerStore.delete(dbKey);
|
||||||
"container doesn't exist.",
|
} else {
|
||||||
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
|
// Where did the container go? o_O
|
||||||
|
LOG.warn("Unable to remove the container {} from container store," +
|
||||||
|
" it's missing!", containerID);
|
||||||
}
|
}
|
||||||
containerStore.delete(dbKey);
|
} catch (ContainerNotFoundException cnfe) {
|
||||||
|
throw new SCMException(
|
||||||
|
"Failed to delete container " + containerID + ", reason : " +
|
||||||
|
"container doesn't exist.",
|
||||||
|
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
|
||||||
} finally {
|
} finally {
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
}
|
}
|
||||||
|
@ -354,40 +352,51 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public HddsProtos.LifeCycleState updateContainerState(
|
public HddsProtos.LifeCycleState updateContainerState(
|
||||||
long containerID, HddsProtos.LifeCycleEvent event) throws
|
ContainerID containerID, HddsProtos.LifeCycleEvent event)
|
||||||
IOException {
|
throws IOException {
|
||||||
ContainerInfo containerInfo;
|
// Should we return the updated ContainerInfo instead of LifeCycleState?
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
byte[] dbKey = Longs.toByteArray(containerID);
|
ContainerInfo updatedContainer =
|
||||||
byte[] containerBytes = containerStore.get(dbKey);
|
updateContainerStateInternal(containerID, event);
|
||||||
if (containerBytes == null) {
|
if (!updatedContainer.isOpen()) {
|
||||||
throw new SCMException(
|
pipelineSelector.removeContainerFromPipeline(
|
||||||
"Failed to update container state"
|
updatedContainer.getPipelineID(), containerID.getId());
|
||||||
+ containerID
|
|
||||||
+ ", reason : container doesn't exist.",
|
|
||||||
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
|
|
||||||
}
|
}
|
||||||
containerInfo =
|
final byte[] dbKey = Longs.toByteArray(containerID.getId());
|
||||||
ContainerInfo.fromProtobuf(HddsProtos.SCMContainerInfo.PARSER
|
containerStore.put(dbKey, updatedContainer.getProtobuf().toByteArray());
|
||||||
.parseFrom(containerBytes));
|
return updatedContainer.getState();
|
||||||
|
} catch (ContainerNotFoundException cnfe) {
|
||||||
|
throw new SCMException(
|
||||||
|
"Failed to update container state"
|
||||||
|
+ containerID
|
||||||
|
+ ", reason : container doesn't exist.",
|
||||||
|
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
Preconditions.checkNotNull(containerInfo);
|
private ContainerInfo updateContainerStateInternal(ContainerID containerID,
|
||||||
|
HddsProtos.LifeCycleEvent event) throws IOException {
|
||||||
|
// Refactor the below code for better clarity.
|
||||||
|
try {
|
||||||
|
final ContainerInfo info =
|
||||||
|
containerStateManager.getContainer(containerID);
|
||||||
switch (event) {
|
switch (event) {
|
||||||
case CREATE:
|
case CREATE:
|
||||||
// Acquire lease on container
|
// Acquire lease on container
|
||||||
Lease<ContainerInfo> containerLease =
|
Lease<ContainerInfo> containerLease =
|
||||||
containerLeaseManager.acquire(containerInfo);
|
containerLeaseManager.acquire(info);
|
||||||
// Register callback to be executed in case of timeout
|
// Register callback to be executed in case of timeout
|
||||||
containerLease.registerCallBack(() -> {
|
containerLease.registerCallBack(() -> {
|
||||||
updateContainerState(containerID,
|
updateContainerState(containerID,
|
||||||
HddsProtos.LifeCycleEvent.TIMEOUT);
|
HddsProtos.LifeCycleEvent.TIMEOUT);
|
||||||
return null;
|
return null; });
|
||||||
});
|
|
||||||
break;
|
break;
|
||||||
case CREATED:
|
case CREATED:
|
||||||
// Release the lease on container
|
// Release the lease on container
|
||||||
containerLeaseManager.release(containerInfo);
|
containerLeaseManager.release(info);
|
||||||
break;
|
break;
|
||||||
case FINALIZE:
|
case FINALIZE:
|
||||||
// TODO: we don't need a lease manager here for closing as the
|
// TODO: we don't need a lease manager here for closing as the
|
||||||
|
@ -412,28 +421,20 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
// If the below updateContainerState call fails, we should revert the
|
// If the below updateContainerState call fails, we should revert the
|
||||||
// changes made in switch case.
|
// changes made in switch case.
|
||||||
// Like releasing the lease in case of BEGIN_CREATE.
|
// Like releasing the lease in case of BEGIN_CREATE.
|
||||||
ContainerInfo updatedContainer = containerStateManager
|
return containerStateManager.updateContainerState(containerID, event);
|
||||||
.updateContainerState(containerInfo, event);
|
|
||||||
if (!updatedContainer.isContainerOpen()) {
|
|
||||||
pipelineSelector.removeContainerFromPipeline(
|
|
||||||
containerInfo.getPipelineID(), containerID);
|
|
||||||
}
|
|
||||||
containerStore.put(dbKey, updatedContainer.getProtobuf().toByteArray());
|
|
||||||
return updatedContainer.getState();
|
|
||||||
} catch (LeaseException e) {
|
} catch (LeaseException e) {
|
||||||
throw new IOException("Lease Exception.", e);
|
throw new IOException("Lease Exception.", e);
|
||||||
} finally {
|
|
||||||
lock.unlock();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Update deleteTransactionId according to deleteTransactionMap.
|
/**
|
||||||
*
|
* Update deleteTransactionId according to deleteTransactionMap.
|
||||||
* @param deleteTransactionMap Maps the containerId to latest delete
|
*
|
||||||
* transaction id for the container.
|
* @param deleteTransactionMap Maps the containerId to latest delete
|
||||||
* @throws IOException
|
* transaction id for the container.
|
||||||
*/
|
* @throws IOException
|
||||||
|
*/
|
||||||
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
|
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (deleteTransactionMap == null) {
|
if (deleteTransactionMap == null) {
|
||||||
|
@ -466,16 +467,6 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the container State Manager.
|
|
||||||
*
|
|
||||||
* @return ContainerStateManager
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public ContainerStateManager getStateManager() {
|
|
||||||
return containerStateManager;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return a container matching the attributes specified.
|
* Return a container matching the attributes specified.
|
||||||
*
|
*
|
||||||
|
@ -489,7 +480,7 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
public ContainerWithPipeline getMatchingContainerWithPipeline(
|
public ContainerWithPipeline getMatchingContainerWithPipeline(
|
||||||
final long sizeRequired, String owner, ReplicationType type,
|
final long sizeRequired, String owner, ReplicationType type,
|
||||||
ReplicationFactor factor, LifeCycleState state) throws IOException {
|
ReplicationFactor factor, LifeCycleState state) throws IOException {
|
||||||
ContainerInfo containerInfo = getStateManager()
|
ContainerInfo containerInfo = containerStateManager
|
||||||
.getMatchingContainer(sizeRequired, owner, type, factor, state);
|
.getMatchingContainer(sizeRequired, owner, type, factor, state);
|
||||||
if (containerInfo == null) {
|
if (containerInfo == null) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -518,70 +509,45 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void processContainerReports(DatanodeDetails datanodeDetails,
|
public void processContainerReports(DatanodeDetails datanodeDetails,
|
||||||
ContainerReportsProto reports, boolean isRegisterCall)
|
ContainerReportsProto reports) throws IOException {
|
||||||
throws IOException {
|
|
||||||
List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
|
List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
|
||||||
containerInfos = reports.getReportsList();
|
containerInfos = reports.getReportsList();
|
||||||
PendingDeleteStatusList pendingDeleteStatusList =
|
PendingDeleteStatusList pendingDeleteStatusList =
|
||||||
new PendingDeleteStatusList(datanodeDetails);
|
new PendingDeleteStatusList(datanodeDetails);
|
||||||
for (StorageContainerDatanodeProtocolProtos.ContainerInfo contInfo :
|
for (StorageContainerDatanodeProtocolProtos.ContainerInfo newInfo :
|
||||||
containerInfos) {
|
containerInfos) {
|
||||||
// Update replica info during registration process.
|
ContainerID id = ContainerID.valueof(newInfo.getContainerID());
|
||||||
if (isRegisterCall) {
|
ContainerReplica replica = ContainerReplica.newBuilder()
|
||||||
try {
|
.setContainerID(id)
|
||||||
getStateManager().addContainerReplica(ContainerID.
|
.setDatanodeDetails(datanodeDetails)
|
||||||
valueof(contInfo.getContainerID()), datanodeDetails);
|
.setOriginNodeId(datanodeDetails.getUuid())
|
||||||
} catch (Exception ex) {
|
.build();
|
||||||
// Continue to next one after logging the error.
|
|
||||||
LOG.error("Error while adding replica for containerId {}.",
|
|
||||||
contInfo.getContainerID(), ex);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
byte[] dbKey = Longs.toByteArray(contInfo.getContainerID());
|
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
byte[] containerBytes = containerStore.get(dbKey);
|
containerStateManager.updateContainerReplica(id, replica);
|
||||||
if (containerBytes != null) {
|
ContainerInfo currentInfo = containerStateManager.getContainer(id);
|
||||||
HddsProtos.SCMContainerInfo knownState =
|
if (newInfo.getState() == LifeCycleState.CLOSING
|
||||||
HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
|
&& currentInfo.getState() == LifeCycleState.CLOSED) {
|
||||||
|
currentInfo = updateContainerStateInternal(id, LifeCycleEvent.CLOSE);
|
||||||
if (knownState.getState() == LifeCycleState.CLOSING
|
|
||||||
&& contInfo.getState() == LifeCycleState.CLOSED) {
|
|
||||||
|
|
||||||
updateContainerState(contInfo.getContainerID(),
|
|
||||||
LifeCycleEvent.CLOSE);
|
|
||||||
|
|
||||||
//reread the container
|
|
||||||
knownState =
|
|
||||||
HddsProtos.SCMContainerInfo.PARSER
|
|
||||||
.parseFrom(containerStore.get(dbKey));
|
|
||||||
}
|
|
||||||
|
|
||||||
HddsProtos.SCMContainerInfo newState =
|
|
||||||
reconcileState(contInfo, knownState, datanodeDetails);
|
|
||||||
|
|
||||||
if (knownState.getDeleteTransactionId() > contInfo
|
|
||||||
.getDeleteTransactionId()) {
|
|
||||||
pendingDeleteStatusList
|
|
||||||
.addPendingDeleteStatus(contInfo.getDeleteTransactionId(),
|
|
||||||
knownState.getDeleteTransactionId(),
|
|
||||||
knownState.getContainerID());
|
|
||||||
}
|
|
||||||
|
|
||||||
// FIX ME: This can be optimized, we write twice to memory, where a
|
|
||||||
// single write would work well.
|
|
||||||
//
|
|
||||||
// We need to write this to DB again since the closed only write
|
|
||||||
// the updated State.
|
|
||||||
containerStore.put(dbKey, newState.toByteArray());
|
|
||||||
|
|
||||||
} else {
|
|
||||||
// Container not found in our container db.
|
|
||||||
LOG.error("Error while processing container report from datanode :" +
|
|
||||||
" {}, for container: {}, reason: container doesn't exist in" +
|
|
||||||
"container database.", datanodeDetails,
|
|
||||||
contInfo.getContainerID());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
HddsProtos.SCMContainerInfo newState =
|
||||||
|
reconcileState(newInfo, currentInfo);
|
||||||
|
|
||||||
|
if (currentInfo.getDeleteTransactionId() >
|
||||||
|
newInfo.getDeleteTransactionId()) {
|
||||||
|
pendingDeleteStatusList
|
||||||
|
.addPendingDeleteStatus(newInfo.getDeleteTransactionId(),
|
||||||
|
currentInfo.getDeleteTransactionId(),
|
||||||
|
currentInfo.getContainerID());
|
||||||
|
}
|
||||||
|
containerStateManager.updateContainerInfo(
|
||||||
|
ContainerInfo.fromProtobuf(newState));
|
||||||
|
containerStore.put(id.getBytes(), newState.toByteArray());
|
||||||
|
} catch (ContainerNotFoundException e) {
|
||||||
|
LOG.error("Error while processing container report from datanode :" +
|
||||||
|
" {}, for container: {}, reason: container doesn't exist in" +
|
||||||
|
"container database.", datanodeDetails, id);
|
||||||
} finally {
|
} finally {
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
}
|
}
|
||||||
|
@ -598,36 +564,21 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
*
|
*
|
||||||
* @param datanodeState - State from the Datanode.
|
* @param datanodeState - State from the Datanode.
|
||||||
* @param knownState - State inside SCM.
|
* @param knownState - State inside SCM.
|
||||||
* @param dnDetails
|
|
||||||
* @return new SCM State for this container.
|
* @return new SCM State for this container.
|
||||||
*/
|
*/
|
||||||
private HddsProtos.SCMContainerInfo reconcileState(
|
private HddsProtos.SCMContainerInfo reconcileState(
|
||||||
StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState,
|
StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState,
|
||||||
SCMContainerInfo knownState, DatanodeDetails dnDetails) {
|
ContainerInfo knownState) {
|
||||||
HddsProtos.SCMContainerInfo.Builder builder =
|
HddsProtos.SCMContainerInfo.Builder builder =
|
||||||
HddsProtos.SCMContainerInfo.newBuilder();
|
HddsProtos.SCMContainerInfo.newBuilder();
|
||||||
builder.setContainerID(knownState.getContainerID())
|
builder.setContainerID(knownState.getContainerID())
|
||||||
.setPipelineID(knownState.getPipelineID())
|
.setPipelineID(knownState.getPipelineID().getProtobuf())
|
||||||
.setReplicationType(knownState.getReplicationType())
|
|
||||||
.setReplicationFactor(knownState.getReplicationFactor());
|
|
||||||
|
|
||||||
// TODO: If current state doesn't have this DN in list of DataNodes with
|
|
||||||
// replica then add it in list of replicas.
|
|
||||||
|
|
||||||
// If used size is greater than allocated size, we will be updating
|
|
||||||
// allocated size with used size. This update is done as a fallback
|
|
||||||
// mechanism in case SCM crashes without properly updating allocated
|
|
||||||
// size. Correct allocated value will be updated by
|
|
||||||
// ContainerStateManager during SCM shutdown.
|
|
||||||
long usedSize = datanodeState.getUsed();
|
|
||||||
long allocated = knownState.getAllocatedBytes() > usedSize ?
|
|
||||||
knownState.getAllocatedBytes() : usedSize;
|
|
||||||
builder.setAllocatedBytes(allocated)
|
|
||||||
.setUsedBytes(usedSize)
|
|
||||||
.setNumberOfKeys(datanodeState.getKeyCount())
|
|
||||||
.setState(knownState.getState())
|
.setState(knownState.getState())
|
||||||
|
.setReplicationType(knownState.getReplicationType())
|
||||||
|
.setReplicationFactor(knownState.getReplicationFactor())
|
||||||
|
.setUsedBytes(datanodeState.getUsed())
|
||||||
|
.setNumberOfKeys(datanodeState.getKeyCount())
|
||||||
.setStateEnterTime(knownState.getStateEnterTime())
|
.setStateEnterTime(knownState.getStateEnterTime())
|
||||||
.setContainerID(knownState.getContainerID())
|
|
||||||
.setDeleteTransactionId(knownState.getDeleteTransactionId());
|
.setDeleteTransactionId(knownState.getDeleteTransactionId());
|
||||||
if (knownState.getOwner() != null) {
|
if (knownState.getOwner() != null) {
|
||||||
builder.setOwner(knownState.getOwner());
|
builder.setOwner(knownState.getOwner());
|
||||||
|
@ -635,20 +586,40 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
return builder.build();
|
return builder.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* In Container is in closed state, if it is in closed, Deleting or Deleted
|
* Returns the latest list of DataNodes where replica for given containerId
|
||||||
* State.
|
* exist. Throws an SCMException if no entry is found for given containerId.
|
||||||
*
|
*
|
||||||
* @param info - ContainerInfo.
|
* @param containerID
|
||||||
* @return true if is in open state, false otherwise
|
* @return Set<DatanodeDetails>
|
||||||
*/
|
*/
|
||||||
private boolean shouldClose(ContainerInfo info) {
|
public Set<ContainerReplica> getContainerReplicas(
|
||||||
return info.getState() == HddsProtos.LifeCycleState.OPEN;
|
final ContainerID containerID) throws ContainerNotFoundException {
|
||||||
|
return containerStateManager.getContainerReplicas(containerID);
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean isClosed(ContainerInfo info) {
|
/**
|
||||||
return info.getState() == HddsProtos.LifeCycleState.CLOSED;
|
* Add a container Replica for given DataNode.
|
||||||
|
*
|
||||||
|
* @param containerID
|
||||||
|
* @param replica
|
||||||
|
*/
|
||||||
|
public void updateContainerReplica(final ContainerID containerID,
|
||||||
|
final ContainerReplica replica) throws ContainerNotFoundException {
|
||||||
|
containerStateManager.updateContainerReplica(containerID, replica);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove a container Replica for given DataNode.
|
||||||
|
*
|
||||||
|
* @param containerID
|
||||||
|
* @param replica
|
||||||
|
* @return True of dataNode is removed successfully else false.
|
||||||
|
*/
|
||||||
|
public void removeContainerReplica(final ContainerID containerID,
|
||||||
|
final ContainerReplica replica)
|
||||||
|
throws ContainerNotFoundException, ContainerReplicaNotFoundException {
|
||||||
|
containerStateManager.removeContainerReplica(containerID, replica);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -671,7 +642,6 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
containerLeaseManager.shutdown();
|
containerLeaseManager.shutdown();
|
||||||
}
|
}
|
||||||
if (containerStateManager != null) {
|
if (containerStateManager != null) {
|
||||||
flushContainerInfo();
|
|
||||||
containerStateManager.close();
|
containerStateManager.close();
|
||||||
}
|
}
|
||||||
if (containerStore != null) {
|
if (containerStore != null) {
|
||||||
|
@ -683,48 +653,6 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Since allocatedBytes of a container is only in memory, stored in
|
|
||||||
* containerStateManager, when closing SCMContainerManager, we need to update
|
|
||||||
* this in the container store.
|
|
||||||
*
|
|
||||||
* @throws IOException on failure.
|
|
||||||
*/
|
|
||||||
@VisibleForTesting
|
|
||||||
public void flushContainerInfo() throws IOException {
|
|
||||||
List<ContainerInfo> containers = containerStateManager.getAllContainers();
|
|
||||||
List<Long> failedContainers = new ArrayList<>();
|
|
||||||
for (ContainerInfo info : containers) {
|
|
||||||
// even if some container updated failed, others can still proceed
|
|
||||||
try {
|
|
||||||
byte[] dbKey = Longs.toByteArray(info.getContainerID());
|
|
||||||
byte[] containerBytes = containerStore.get(dbKey);
|
|
||||||
// TODO : looks like when a container is deleted, the container is
|
|
||||||
// removed from containerStore but not containerStateManager, so it can
|
|
||||||
// return info of a deleted container. may revisit this in the future,
|
|
||||||
// for now, just skip a not-found container
|
|
||||||
if (containerBytes != null) {
|
|
||||||
containerStore.put(dbKey, info.getProtobuf().toByteArray());
|
|
||||||
} else {
|
|
||||||
LOG.debug("Container state manager has container {} but not found " +
|
|
||||||
"in container store, a deleted container?",
|
|
||||||
info.getContainerID());
|
|
||||||
}
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
failedContainers.add(info.getContainerID());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (!failedContainers.isEmpty()) {
|
|
||||||
throw new IOException("Error in flushing container info from container " +
|
|
||||||
"state manager: " + failedContainers);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
public MetadataStore getContainerStore() {
|
|
||||||
return containerStore;
|
|
||||||
}
|
|
||||||
|
|
||||||
public PipelineSelector getPipelineSelector() {
|
public PipelineSelector getPipelineSelector() {
|
||||||
return pipelineSelector;
|
return pipelineSelector;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,12 +22,14 @@ import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ThreadFactory;
|
import java.util.concurrent.ThreadFactory;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
|
||||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
||||||
.ContainerPlacementPolicy;
|
.ContainerPlacementPolicy;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
|
@ -64,14 +66,14 @@ public class ReplicationManager implements Runnable {
|
||||||
|
|
||||||
private boolean running = true;
|
private boolean running = true;
|
||||||
|
|
||||||
private ContainerStateManager containerStateManager;
|
private ContainerManager containerManager;
|
||||||
|
|
||||||
public ReplicationManager(ContainerPlacementPolicy containerPlacement,
|
public ReplicationManager(ContainerPlacementPolicy containerPlacement,
|
||||||
ContainerStateManager containerStateManager, EventQueue eventQueue,
|
ContainerManager containerManager, EventQueue eventQueue,
|
||||||
LeaseManager<Long> commandWatcherLeaseManager) {
|
LeaseManager<Long> commandWatcherLeaseManager) {
|
||||||
|
|
||||||
this.containerPlacement = containerPlacement;
|
this.containerPlacement = containerPlacement;
|
||||||
this.containerStateManager = containerStateManager;
|
this.containerManager = containerManager;
|
||||||
this.eventPublisher = eventQueue;
|
this.eventPublisher = eventQueue;
|
||||||
|
|
||||||
this.replicationCommandWatcher =
|
this.replicationCommandWatcher =
|
||||||
|
@ -106,7 +108,7 @@ public class ReplicationManager implements Runnable {
|
||||||
|
|
||||||
ContainerID containerID = new ContainerID(request.getContainerId());
|
ContainerID containerID = new ContainerID(request.getContainerId());
|
||||||
ContainerInfo containerInfo =
|
ContainerInfo containerInfo =
|
||||||
containerStateManager.getContainer(containerID);
|
containerManager.getContainer(containerID);
|
||||||
|
|
||||||
Preconditions.checkNotNull(containerInfo,
|
Preconditions.checkNotNull(containerInfo,
|
||||||
"No information about the container " + request.getContainerId());
|
"No information about the container " + request.getContainerId());
|
||||||
|
@ -116,10 +118,10 @@ public class ReplicationManager implements Runnable {
|
||||||
"Container should be in closed state");
|
"Container should be in closed state");
|
||||||
|
|
||||||
//check the current replication
|
//check the current replication
|
||||||
List<DatanodeDetails> datanodesWithReplicas =
|
List<ContainerReplica> containerReplicas =
|
||||||
new ArrayList<>(getCurrentReplicas(request));
|
new ArrayList<>(getCurrentReplicas(request));
|
||||||
|
|
||||||
if (datanodesWithReplicas.size() == 0) {
|
if (containerReplicas.size() == 0) {
|
||||||
LOG.warn(
|
LOG.warn(
|
||||||
"Container {} should be replicated but can't find any existing "
|
"Container {} should be replicated but can't find any existing "
|
||||||
+ "replicas",
|
+ "replicas",
|
||||||
|
@ -134,21 +136,23 @@ public class ReplicationManager implements Runnable {
|
||||||
.size();
|
.size();
|
||||||
|
|
||||||
int deficit =
|
int deficit =
|
||||||
request.getExpecReplicationCount() - datanodesWithReplicas.size()
|
request.getExpecReplicationCount() - containerReplicas.size()
|
||||||
- inFlightReplications;
|
- inFlightReplications;
|
||||||
|
|
||||||
if (deficit > 0) {
|
if (deficit > 0) {
|
||||||
|
|
||||||
|
List<DatanodeDetails> datanodes = containerReplicas.stream()
|
||||||
|
.map(ContainerReplica::getDatanodeDetails)
|
||||||
|
.collect(Collectors.toList());
|
||||||
List<DatanodeDetails> selectedDatanodes = containerPlacement
|
List<DatanodeDetails> selectedDatanodes = containerPlacement
|
||||||
.chooseDatanodes(datanodesWithReplicas, deficit,
|
.chooseDatanodes(datanodes, deficit,
|
||||||
containerInfo.getUsedBytes());
|
containerInfo.getUsedBytes());
|
||||||
|
|
||||||
//send the command
|
//send the command
|
||||||
for (DatanodeDetails datanode : selectedDatanodes) {
|
for (DatanodeDetails datanode : selectedDatanodes) {
|
||||||
|
|
||||||
ReplicateContainerCommand replicateCommand =
|
ReplicateContainerCommand replicateCommand =
|
||||||
new ReplicateContainerCommand(containerID.getId(),
|
new ReplicateContainerCommand(containerID.getId(), datanodes);
|
||||||
datanodesWithReplicas);
|
|
||||||
|
|
||||||
eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND,
|
eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND,
|
||||||
new CommandForDatanode<>(
|
new CommandForDatanode<>(
|
||||||
|
@ -174,9 +178,9 @@ public class ReplicationManager implements Runnable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
protected Set<DatanodeDetails> getCurrentReplicas(ReplicationRequest request)
|
protected Set<ContainerReplica> getCurrentReplicas(ReplicationRequest request)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return containerStateManager
|
return containerManager
|
||||||
.getContainerReplicas(new ContainerID(request.getContainerId()));
|
.getContainerReplicas(new ContainerID(request.getContainerId()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -234,7 +238,11 @@ public class ReplicationManager implements Runnable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class ReplicationCompleted implements IdentifiableEventPayload {
|
/**
|
||||||
|
* Add javadoc.
|
||||||
|
*/
|
||||||
|
public static class ReplicationCompleted
|
||||||
|
implements IdentifiableEventPayload {
|
||||||
|
|
||||||
private final long uuid;
|
private final long uuid;
|
||||||
|
|
||||||
|
|
|
@ -20,19 +20,21 @@ package org.apache.hadoop.hdds.scm.container.states;
|
||||||
|
|
||||||
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 java.util.HashSet;
|
|
||||||
import java.util.Set;
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
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.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -46,8 +48,6 @@ import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
||||||
.CONTAINER_EXISTS;
|
.CONTAINER_EXISTS;
|
||||||
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
||||||
.FAILED_TO_CHANGE_CONTAINER_STATE;
|
.FAILED_TO_CHANGE_CONTAINER_STATE;
|
||||||
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
|
||||||
.FAILED_TO_FIND_CONTAINER;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Container State Map acts like a unified map for various attributes that are
|
* Container State Map acts like a unified map for various attributes that are
|
||||||
|
@ -84,16 +84,15 @@ public class ContainerStateMap {
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(ContainerStateMap.class);
|
LoggerFactory.getLogger(ContainerStateMap.class);
|
||||||
|
|
||||||
|
private final static NavigableSet<ContainerID> EMPTY_SET =
|
||||||
|
Collections.unmodifiableNavigableSet(new TreeSet<>());
|
||||||
|
|
||||||
private final ContainerAttribute<LifeCycleState> lifeCycleStateMap;
|
private final ContainerAttribute<LifeCycleState> lifeCycleStateMap;
|
||||||
private final ContainerAttribute<String> ownerMap;
|
private final ContainerAttribute<String> ownerMap;
|
||||||
private final ContainerAttribute<ReplicationFactor> factorMap;
|
private final ContainerAttribute<ReplicationFactor> factorMap;
|
||||||
private final ContainerAttribute<ReplicationType> typeMap;
|
private final ContainerAttribute<ReplicationType> typeMap;
|
||||||
|
|
||||||
private final Map<ContainerID, ContainerInfo> containerMap;
|
private final Map<ContainerID, ContainerInfo> containerMap;
|
||||||
// Map to hold replicas of given container.
|
private final Map<ContainerID, Set<ContainerReplica>> replicaMap;
|
||||||
private final Map<ContainerID, Set<DatanodeDetails>> contReplicaMap;
|
|
||||||
private final static NavigableSet<ContainerID> EMPTY_SET =
|
|
||||||
Collections.unmodifiableNavigableSet(new TreeSet<>());
|
|
||||||
private final Map<ContainerQueryKey, NavigableSet<ContainerID>> resultCache;
|
private final Map<ContainerQueryKey, NavigableSet<ContainerID>> resultCache;
|
||||||
|
|
||||||
// Container State Map lock should be held before calling into
|
// Container State Map lock should be held before calling into
|
||||||
|
@ -105,18 +104,14 @@ public class ContainerStateMap {
|
||||||
* Create a ContainerStateMap.
|
* Create a ContainerStateMap.
|
||||||
*/
|
*/
|
||||||
public ContainerStateMap() {
|
public ContainerStateMap() {
|
||||||
lifeCycleStateMap = new ContainerAttribute<>();
|
this.lifeCycleStateMap = new ContainerAttribute<>();
|
||||||
ownerMap = new ContainerAttribute<>();
|
this.ownerMap = new ContainerAttribute<>();
|
||||||
factorMap = new ContainerAttribute<>();
|
this.factorMap = new ContainerAttribute<>();
|
||||||
typeMap = new ContainerAttribute<>();
|
this.typeMap = new ContainerAttribute<>();
|
||||||
containerMap = new HashMap<>();
|
this.containerMap = new HashMap<>();
|
||||||
lock = new ReentrantReadWriteLock();
|
this.lock = new ReentrantReadWriteLock();
|
||||||
contReplicaMap = new HashMap<>();
|
this.replicaMap = new HashMap<>();
|
||||||
// new InstrumentedLock(getClass().getName(), LOG,
|
this.resultCache = new ConcurrentHashMap<>();
|
||||||
// new ReentrantLock(),
|
|
||||||
// 1000,
|
|
||||||
// 300));
|
|
||||||
resultCache = new ConcurrentHashMap<>();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -125,7 +120,7 @@ public class ContainerStateMap {
|
||||||
* @param info - container info
|
* @param info - container info
|
||||||
* @throws SCMException - throws if create failed.
|
* @throws SCMException - throws if create failed.
|
||||||
*/
|
*/
|
||||||
public void addContainer(ContainerInfo info)
|
public void addContainer(final ContainerInfo info)
|
||||||
throws SCMException {
|
throws SCMException {
|
||||||
Preconditions.checkNotNull(info, "Container Info cannot be null");
|
Preconditions.checkNotNull(info, "Container Info cannot be null");
|
||||||
Preconditions.checkArgument(info.getReplicationFactor().getNumber() > 0,
|
Preconditions.checkArgument(info.getReplicationFactor().getNumber() > 0,
|
||||||
|
@ -133,7 +128,7 @@ public class ContainerStateMap {
|
||||||
|
|
||||||
lock.writeLock().lock();
|
lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
ContainerID id = ContainerID.valueof(info.getContainerID());
|
final ContainerID id = info.containerID();
|
||||||
if (containerMap.putIfAbsent(id, info) != null) {
|
if (containerMap.putIfAbsent(id, info) != null) {
|
||||||
LOG.debug("Duplicate container ID detected. {}", id);
|
LOG.debug("Duplicate container ID detected. {}", id);
|
||||||
throw new
|
throw new
|
||||||
|
@ -145,6 +140,7 @@ public class ContainerStateMap {
|
||||||
ownerMap.insert(info.getOwner(), id);
|
ownerMap.insert(info.getOwner(), id);
|
||||||
factorMap.insert(info.getReplicationFactor(), id);
|
factorMap.insert(info.getReplicationFactor(), id);
|
||||||
typeMap.insert(info.getReplicationType(), id);
|
typeMap.insert(info.getReplicationType(), id);
|
||||||
|
replicaMap.put(id, new HashSet<>());
|
||||||
|
|
||||||
// Flush the cache of this container type, will be added later when
|
// Flush the cache of this container type, will be added later when
|
||||||
// get container queries are executed.
|
// get container queries are executed.
|
||||||
|
@ -156,23 +152,30 @@ public class ContainerStateMap {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the latest state of Container from SCM's Container State Map.
|
* Removes a Container Entry from ContainerStateMap.
|
||||||
*
|
*
|
||||||
* @param info - ContainerInfo
|
* @param containerID - ContainerID
|
||||||
* @return ContainerInfo
|
* @throws SCMException - throws if create failed.
|
||||||
*/
|
*/
|
||||||
public ContainerInfo getContainerInfo(ContainerInfo info) {
|
public void removeContainer(final ContainerID containerID)
|
||||||
return getContainerInfo(info.getContainerID());
|
throws ContainerNotFoundException {
|
||||||
}
|
Preconditions.checkNotNull(containerID, "ContainerID cannot be null");
|
||||||
|
lock.writeLock().lock();
|
||||||
/**
|
try {
|
||||||
* Returns the latest state of Container from SCM's Container State Map.
|
checkIfContainerExist(containerID);
|
||||||
*
|
// Should we revert back to the original state if any of the below
|
||||||
* @param containerID - int
|
// remove operation fails?
|
||||||
* @return container info, if found.
|
final ContainerInfo info = containerMap.remove(containerID);
|
||||||
*/
|
lifeCycleStateMap.remove(info.getState(), containerID);
|
||||||
public ContainerInfo getContainerInfo(long containerID) {
|
ownerMap.remove(info.getOwner(), containerID);
|
||||||
return getContainerInfo(ContainerID.valueof(containerID));
|
factorMap.remove(info.getReplicationFactor(), containerID);
|
||||||
|
typeMap.remove(info.getReplicationType(), containerID);
|
||||||
|
// Flush the cache of this container type.
|
||||||
|
flushCache(info);
|
||||||
|
LOG.trace("Removed container with {} successfully.", containerID);
|
||||||
|
} finally {
|
||||||
|
lock.writeLock().unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -181,9 +184,11 @@ public class ContainerStateMap {
|
||||||
* @param containerID - ContainerID
|
* @param containerID - ContainerID
|
||||||
* @return container info, if found.
|
* @return container info, if found.
|
||||||
*/
|
*/
|
||||||
public ContainerInfo getContainerInfo(ContainerID containerID) {
|
public ContainerInfo getContainerInfo(final ContainerID containerID)
|
||||||
|
throws ContainerNotFoundException {
|
||||||
lock.readLock().lock();
|
lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
|
checkIfContainerExist(containerID);
|
||||||
return containerMap.get(containerID);
|
return containerMap.get(containerID);
|
||||||
} finally {
|
} finally {
|
||||||
lock.readLock().unlock();
|
lock.readLock().unlock();
|
||||||
|
@ -197,21 +202,17 @@ public class ContainerStateMap {
|
||||||
* @param containerID
|
* @param containerID
|
||||||
* @return Set<DatanodeDetails>
|
* @return Set<DatanodeDetails>
|
||||||
*/
|
*/
|
||||||
public Set<DatanodeDetails> getContainerReplicas(ContainerID containerID)
|
public Set<ContainerReplica> getContainerReplicas(
|
||||||
throws SCMException {
|
final ContainerID containerID) throws ContainerNotFoundException {
|
||||||
Preconditions.checkNotNull(containerID);
|
Preconditions.checkNotNull(containerID);
|
||||||
lock.readLock().lock();
|
lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
if (contReplicaMap.containsKey(containerID)) {
|
checkIfContainerExist(containerID);
|
||||||
return Collections
|
return Collections
|
||||||
.unmodifiableSet(contReplicaMap.get(containerID));
|
.unmodifiableSet(new HashSet<>(replicaMap.get(containerID)));
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
lock.readLock().unlock();
|
lock.readLock().unlock();
|
||||||
}
|
}
|
||||||
throw new SCMException(
|
|
||||||
"No entry exist for containerId: " + containerID + " in replica map.",
|
|
||||||
ResultCodes.NO_REPLICA_FOUND);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -220,26 +221,17 @@ public class ContainerStateMap {
|
||||||
* ContainerId.
|
* ContainerId.
|
||||||
*
|
*
|
||||||
* @param containerID
|
* @param containerID
|
||||||
* @param dnList
|
* @param replica
|
||||||
*/
|
*/
|
||||||
public void addContainerReplica(ContainerID containerID,
|
public void updateContainerReplica(final ContainerID containerID,
|
||||||
DatanodeDetails... dnList) {
|
final ContainerReplica replica) throws ContainerNotFoundException {
|
||||||
Preconditions.checkNotNull(containerID);
|
Preconditions.checkNotNull(containerID);
|
||||||
lock.writeLock().lock();
|
lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
for (DatanodeDetails dn : dnList) {
|
checkIfContainerExist(containerID);
|
||||||
Preconditions.checkNotNull(dn);
|
Set<ContainerReplica> replicas = replicaMap.get(containerID);
|
||||||
if (contReplicaMap.containsKey(containerID)) {
|
replicas.remove(replica);
|
||||||
if(!contReplicaMap.get(containerID).add(dn)) {
|
replicas.add(replica);
|
||||||
LOG.debug("ReplicaMap already contains entry for container Id: "
|
|
||||||
+ "{},DataNode: {}", containerID, dn);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
Set<DatanodeDetails> dnSet = new HashSet<>();
|
|
||||||
dnSet.add(dn);
|
|
||||||
contReplicaMap.put(containerID, dnSet);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
lock.writeLock().unlock();
|
lock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
|
@ -249,61 +241,45 @@ public class ContainerStateMap {
|
||||||
* Remove a container Replica for given DataNode.
|
* Remove a container Replica for given DataNode.
|
||||||
*
|
*
|
||||||
* @param containerID
|
* @param containerID
|
||||||
* @param dn
|
* @param replica
|
||||||
* @return True of dataNode is removed successfully else false.
|
* @return True of dataNode is removed successfully else false.
|
||||||
*/
|
*/
|
||||||
public boolean removeContainerReplica(ContainerID containerID,
|
public void removeContainerReplica(final ContainerID containerID,
|
||||||
DatanodeDetails dn) throws SCMException {
|
final ContainerReplica replica)
|
||||||
|
throws ContainerNotFoundException, ContainerReplicaNotFoundException {
|
||||||
Preconditions.checkNotNull(containerID);
|
Preconditions.checkNotNull(containerID);
|
||||||
Preconditions.checkNotNull(dn);
|
Preconditions.checkNotNull(replica);
|
||||||
|
|
||||||
lock.writeLock().lock();
|
lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
if (contReplicaMap.containsKey(containerID)) {
|
checkIfContainerExist(containerID);
|
||||||
return contReplicaMap.get(containerID).remove(dn);
|
if(!replicaMap.get(containerID).remove(replica)) {
|
||||||
|
throw new ContainerReplicaNotFoundException(
|
||||||
|
"Container #"
|
||||||
|
+ containerID.getId() + ", replica: " + replica);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
lock.writeLock().unlock();
|
lock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
throw new SCMException(
|
|
||||||
"No entry exist for containerId: " + containerID + " in replica map.",
|
|
||||||
ResultCodes.FAILED_TO_FIND_CONTAINER);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
|
// TODO: fix the test case and remove this method!
|
||||||
public static Logger getLOG() {
|
public static Logger getLOG() {
|
||||||
return LOG;
|
return LOG;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the full container Map.
|
|
||||||
*
|
|
||||||
* @return - Map
|
|
||||||
*/
|
|
||||||
public Map<ContainerID, ContainerInfo> getContainerMap() {
|
|
||||||
lock.readLock().lock();
|
|
||||||
try {
|
|
||||||
return Collections.unmodifiableMap(containerMap);
|
|
||||||
} finally {
|
|
||||||
lock.readLock().unlock();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Just update the container State.
|
* Just update the container State.
|
||||||
* @param info ContainerInfo.
|
* @param info ContainerInfo.
|
||||||
*/
|
*/
|
||||||
public void updateContainerInfo(ContainerInfo info) throws SCMException {
|
public void updateContainerInfo(final ContainerInfo info)
|
||||||
Preconditions.checkNotNull(info);
|
throws ContainerNotFoundException {
|
||||||
ContainerInfo currentInfo = null;
|
|
||||||
lock.writeLock().lock();
|
lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
currentInfo = containerMap.get(
|
Preconditions.checkNotNull(info);
|
||||||
ContainerID.valueof(info.getContainerID()));
|
checkIfContainerExist(info.containerID());
|
||||||
|
final ContainerInfo currentInfo = containerMap.get(info.containerID());
|
||||||
if (currentInfo == null) {
|
|
||||||
throw new SCMException("No such container.", FAILED_TO_FIND_CONTAINER);
|
|
||||||
}
|
|
||||||
flushCache(info, currentInfo);
|
flushCache(info, currentInfo);
|
||||||
containerMap.put(info.containerID(), info);
|
containerMap.put(info.containerID(), info);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -314,33 +290,23 @@ public class ContainerStateMap {
|
||||||
/**
|
/**
|
||||||
* Update the State of a container.
|
* Update the State of a container.
|
||||||
*
|
*
|
||||||
* @param info - ContainerInfo
|
* @param containerID - ContainerID
|
||||||
* @param currentState - CurrentState
|
* @param currentState - CurrentState
|
||||||
* @param newState - NewState.
|
* @param newState - NewState.
|
||||||
* @throws SCMException - in case of failure.
|
* @throws SCMException - in case of failure.
|
||||||
*/
|
*/
|
||||||
public void updateState(ContainerInfo info, LifeCycleState currentState,
|
public void updateState(ContainerID containerID, LifeCycleState currentState,
|
||||||
LifeCycleState newState) throws SCMException {
|
LifeCycleState newState) throws SCMException, ContainerNotFoundException {
|
||||||
Preconditions.checkNotNull(currentState);
|
Preconditions.checkNotNull(currentState);
|
||||||
Preconditions.checkNotNull(newState);
|
Preconditions.checkNotNull(newState);
|
||||||
|
|
||||||
ContainerID id = new ContainerID(info.getContainerID());
|
|
||||||
ContainerInfo currentInfo = null;
|
|
||||||
|
|
||||||
lock.writeLock().lock();
|
lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
|
checkIfContainerExist(containerID);
|
||||||
|
final ContainerInfo currentInfo = containerMap.get(containerID);
|
||||||
try {
|
try {
|
||||||
// Just flush both old and new data sets from the result cache.
|
final ContainerInfo newInfo = new ContainerInfo(currentInfo);
|
||||||
ContainerInfo newInfo = new ContainerInfo(info);
|
|
||||||
newInfo.setState(newState);
|
newInfo.setState(newState);
|
||||||
flushCache(newInfo, info);
|
|
||||||
|
|
||||||
currentInfo = containerMap.get(id);
|
|
||||||
|
|
||||||
if (currentInfo == null) {
|
|
||||||
throw new
|
|
||||||
SCMException("No such container.", FAILED_TO_FIND_CONTAINER);
|
|
||||||
}
|
|
||||||
// We are updating two places before this update is done, these can
|
// We are updating two places before this update is done, these can
|
||||||
// fail independently, since the code needs to handle it.
|
// fail independently, since the code needs to handle it.
|
||||||
|
|
||||||
|
@ -351,11 +317,13 @@ public class ContainerStateMap {
|
||||||
// roll back the earlier change we did. If the rollback fails, we can
|
// roll back the earlier change we did. If the rollback fails, we can
|
||||||
// be in an inconsistent state,
|
// be in an inconsistent state,
|
||||||
|
|
||||||
info.setState(newState);
|
containerMap.put(containerID, newInfo);
|
||||||
containerMap.put(id, info);
|
lifeCycleStateMap.update(currentState, newState, containerID);
|
||||||
lifeCycleStateMap.update(currentState, newState, id);
|
|
||||||
LOG.trace("Updated the container {} to new state. Old = {}, new = " +
|
LOG.trace("Updated the container {} to new state. Old = {}, new = " +
|
||||||
"{}", id, currentState, newState);
|
"{}", containerID, currentState, newState);
|
||||||
|
|
||||||
|
// Just flush both old and new data sets from the result cache.
|
||||||
|
flushCache(currentInfo, newInfo);
|
||||||
} catch (SCMException ex) {
|
} catch (SCMException ex) {
|
||||||
LOG.error("Unable to update the container state. {}", ex);
|
LOG.error("Unable to update the container state. {}", ex);
|
||||||
// we need to revert the change in this attribute since we are not
|
// we need to revert the change in this attribute since we are not
|
||||||
|
@ -364,13 +332,13 @@ public class ContainerStateMap {
|
||||||
"old state. Old = {}, Attempted state = {}", currentState,
|
"old state. Old = {}, Attempted state = {}", currentState,
|
||||||
newState);
|
newState);
|
||||||
|
|
||||||
containerMap.put(id, currentInfo);
|
containerMap.put(containerID, currentInfo);
|
||||||
|
|
||||||
// if this line throws, the state map can be in an inconsistent
|
// if this line throws, the state map can be in an inconsistent
|
||||||
// state, since we will have modified the attribute by the
|
// state, since we will have modified the attribute by the
|
||||||
// container state will not in sync since we were not able to put
|
// container state will not in sync since we were not able to put
|
||||||
// that into the hash table.
|
// that into the hash table.
|
||||||
lifeCycleStateMap.update(newState, currentState, id);
|
lifeCycleStateMap.update(newState, currentState, containerID);
|
||||||
|
|
||||||
throw new SCMException("Updating the container map failed.", ex,
|
throw new SCMException("Updating the container map failed.", ex,
|
||||||
FAILED_TO_CHANGE_CONTAINER_STATE);
|
FAILED_TO_CHANGE_CONTAINER_STATE);
|
||||||
|
@ -380,13 +348,17 @@ public class ContainerStateMap {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Set<ContainerID> getAllContainerIDs() {
|
||||||
|
return containerMap.keySet();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns A list of containers owned by a name service.
|
* Returns A list of containers owned by a name service.
|
||||||
*
|
*
|
||||||
* @param ownerName - Name of the NameService.
|
* @param ownerName - Name of the NameService.
|
||||||
* @return - NavigableSet of ContainerIDs.
|
* @return - NavigableSet of ContainerIDs.
|
||||||
*/
|
*/
|
||||||
NavigableSet<ContainerID> getContainerIDsByOwner(String ownerName) {
|
NavigableSet<ContainerID> getContainerIDsByOwner(final String ownerName) {
|
||||||
Preconditions.checkNotNull(ownerName);
|
Preconditions.checkNotNull(ownerName);
|
||||||
lock.readLock().lock();
|
lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
|
@ -402,7 +374,7 @@ public class ContainerStateMap {
|
||||||
* @param type - Replication type -- StandAlone, Ratis etc.
|
* @param type - Replication type -- StandAlone, Ratis etc.
|
||||||
* @return NavigableSet
|
* @return NavigableSet
|
||||||
*/
|
*/
|
||||||
NavigableSet<ContainerID> getContainerIDsByType(ReplicationType type) {
|
NavigableSet<ContainerID> getContainerIDsByType(final ReplicationType type) {
|
||||||
Preconditions.checkNotNull(type);
|
Preconditions.checkNotNull(type);
|
||||||
lock.readLock().lock();
|
lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
|
@ -418,7 +390,8 @@ public class ContainerStateMap {
|
||||||
* @param factor - Replication Factor.
|
* @param factor - Replication Factor.
|
||||||
* @return NavigableSet.
|
* @return NavigableSet.
|
||||||
*/
|
*/
|
||||||
NavigableSet<ContainerID> getContainerIDsByFactor(ReplicationFactor factor) {
|
NavigableSet<ContainerID> getContainerIDsByFactor(
|
||||||
|
final ReplicationFactor factor) {
|
||||||
Preconditions.checkNotNull(factor);
|
Preconditions.checkNotNull(factor);
|
||||||
lock.readLock().lock();
|
lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
|
@ -435,7 +408,7 @@ public class ContainerStateMap {
|
||||||
* @return List of containers by state.
|
* @return List of containers by state.
|
||||||
*/
|
*/
|
||||||
public NavigableSet<ContainerID> getContainerIDsByState(
|
public NavigableSet<ContainerID> getContainerIDsByState(
|
||||||
LifeCycleState state) {
|
final LifeCycleState state) {
|
||||||
Preconditions.checkNotNull(state);
|
Preconditions.checkNotNull(state);
|
||||||
lock.readLock().lock();
|
lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
|
@ -455,8 +428,8 @@ public class ContainerStateMap {
|
||||||
* @return ContainerInfo or Null if not container satisfies the criteria.
|
* @return ContainerInfo or Null if not container satisfies the criteria.
|
||||||
*/
|
*/
|
||||||
public NavigableSet<ContainerID> getMatchingContainerIDs(
|
public NavigableSet<ContainerID> getMatchingContainerIDs(
|
||||||
LifeCycleState state, String owner,
|
final LifeCycleState state, final String owner,
|
||||||
ReplicationFactor factor, ReplicationType type) {
|
final ReplicationFactor factor, final ReplicationType type) {
|
||||||
|
|
||||||
Preconditions.checkNotNull(state, "State cannot be null");
|
Preconditions.checkNotNull(state, "State cannot be null");
|
||||||
Preconditions.checkNotNull(owner, "Owner cannot be null");
|
Preconditions.checkNotNull(owner, "Owner cannot be null");
|
||||||
|
@ -465,7 +438,7 @@ public class ContainerStateMap {
|
||||||
|
|
||||||
lock.readLock().lock();
|
lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
ContainerQueryKey queryKey =
|
final ContainerQueryKey queryKey =
|
||||||
new ContainerQueryKey(state, owner, factor, type);
|
new ContainerQueryKey(state, owner, factor, type);
|
||||||
if(resultCache.containsKey(queryKey)){
|
if(resultCache.containsKey(queryKey)){
|
||||||
return resultCache.get(queryKey);
|
return resultCache.get(queryKey);
|
||||||
|
@ -474,30 +447,33 @@ public class ContainerStateMap {
|
||||||
// If we cannot meet any one condition we return EMPTY_SET immediately.
|
// If we cannot meet any one condition we return EMPTY_SET immediately.
|
||||||
// Since when we intersect these sets, the result will be empty if any
|
// Since when we intersect these sets, the result will be empty if any
|
||||||
// one is empty.
|
// one is empty.
|
||||||
NavigableSet<ContainerID> stateSet =
|
final NavigableSet<ContainerID> stateSet =
|
||||||
lifeCycleStateMap.getCollection(state);
|
lifeCycleStateMap.getCollection(state);
|
||||||
if (stateSet.size() == 0) {
|
if (stateSet.size() == 0) {
|
||||||
return EMPTY_SET;
|
return EMPTY_SET;
|
||||||
}
|
}
|
||||||
|
|
||||||
NavigableSet<ContainerID> ownerSet = ownerMap.getCollection(owner);
|
final NavigableSet<ContainerID> ownerSet =
|
||||||
|
ownerMap.getCollection(owner);
|
||||||
if (ownerSet.size() == 0) {
|
if (ownerSet.size() == 0) {
|
||||||
return EMPTY_SET;
|
return EMPTY_SET;
|
||||||
}
|
}
|
||||||
|
|
||||||
NavigableSet<ContainerID> factorSet = factorMap.getCollection(factor);
|
final NavigableSet<ContainerID> factorSet =
|
||||||
|
factorMap.getCollection(factor);
|
||||||
if (factorSet.size() == 0) {
|
if (factorSet.size() == 0) {
|
||||||
return EMPTY_SET;
|
return EMPTY_SET;
|
||||||
}
|
}
|
||||||
|
|
||||||
NavigableSet<ContainerID> typeSet = typeMap.getCollection(type);
|
final NavigableSet<ContainerID> typeSet =
|
||||||
|
typeMap.getCollection(type);
|
||||||
if (typeSet.size() == 0) {
|
if (typeSet.size() == 0) {
|
||||||
return EMPTY_SET;
|
return EMPTY_SET;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
// if we add more constraints we will just add those sets here..
|
// if we add more constraints we will just add those sets here..
|
||||||
NavigableSet<ContainerID>[] sets = sortBySize(stateSet,
|
final NavigableSet<ContainerID>[] sets = sortBySize(stateSet,
|
||||||
ownerSet, factorSet, typeSet);
|
ownerSet, factorSet, typeSet);
|
||||||
|
|
||||||
NavigableSet<ContainerID> currentSet = sets[0];
|
NavigableSet<ContainerID> currentSet = sets[0];
|
||||||
|
@ -521,12 +497,12 @@ public class ContainerStateMap {
|
||||||
* @return resultSet which is the intersection of these two sets.
|
* @return resultSet which is the intersection of these two sets.
|
||||||
*/
|
*/
|
||||||
private NavigableSet<ContainerID> intersectSets(
|
private NavigableSet<ContainerID> intersectSets(
|
||||||
NavigableSet<ContainerID> smaller,
|
final NavigableSet<ContainerID> smaller,
|
||||||
NavigableSet<ContainerID> bigger) {
|
final NavigableSet<ContainerID> bigger) {
|
||||||
Preconditions.checkState(smaller.size() <= bigger.size(),
|
Preconditions.checkState(smaller.size() <= bigger.size(),
|
||||||
"This function assumes the first set is lesser or equal to second " +
|
"This function assumes the first set is lesser or equal to second " +
|
||||||
"set");
|
"set");
|
||||||
NavigableSet<ContainerID> resultSet = new TreeSet<>();
|
final NavigableSet<ContainerID> resultSet = new TreeSet<>();
|
||||||
for (ContainerID id : smaller) {
|
for (ContainerID id : smaller) {
|
||||||
if (bigger.contains(id)) {
|
if (bigger.contains(id)) {
|
||||||
resultSet.add(id);
|
resultSet.add(id);
|
||||||
|
@ -544,11 +520,11 @@ public class ContainerStateMap {
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private NavigableSet<ContainerID>[] sortBySize(
|
private NavigableSet<ContainerID>[] sortBySize(
|
||||||
NavigableSet<ContainerID>... sets) {
|
final NavigableSet<ContainerID>... sets) {
|
||||||
for (int x = 0; x < sets.length - 1; x++) {
|
for (int x = 0; x < sets.length - 1; x++) {
|
||||||
for (int y = 0; y < sets.length - x - 1; y++) {
|
for (int y = 0; y < sets.length - x - 1; y++) {
|
||||||
if (sets[y].size() > sets[y + 1].size()) {
|
if (sets[y].size() > sets[y + 1].size()) {
|
||||||
NavigableSet temp = sets[y];
|
final NavigableSet temp = sets[y];
|
||||||
sets[y] = sets[y + 1];
|
sets[y] = sets[y + 1];
|
||||||
sets[y + 1] = temp;
|
sets[y + 1] = temp;
|
||||||
}
|
}
|
||||||
|
@ -557,13 +533,22 @@ public class ContainerStateMap {
|
||||||
return sets;
|
return sets;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void flushCache(ContainerInfo... containerInfos) {
|
private void flushCache(final ContainerInfo... containerInfos) {
|
||||||
for (ContainerInfo containerInfo : containerInfos) {
|
for (ContainerInfo containerInfo : containerInfos) {
|
||||||
ContainerQueryKey key = new ContainerQueryKey(containerInfo.getState(),
|
final ContainerQueryKey key = new ContainerQueryKey(
|
||||||
containerInfo.getOwner(), containerInfo.getReplicationFactor(),
|
containerInfo.getState(),
|
||||||
|
containerInfo.getOwner(),
|
||||||
|
containerInfo.getReplicationFactor(),
|
||||||
containerInfo.getReplicationType());
|
containerInfo.getReplicationType());
|
||||||
resultCache.remove(key);
|
resultCache.remove(key);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void checkIfContainerExist(ContainerID containerID)
|
||||||
|
throws ContainerNotFoundException {
|
||||||
|
if (!containerMap.containsKey(containerID)) {
|
||||||
|
throw new ContainerNotFoundException("#" + containerID.getId());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,11 +21,14 @@ package org.apache.hadoop.hdds.scm.node;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerException;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
|
||||||
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
||||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
|
||||||
|
@ -37,7 +40,7 @@ import org.slf4j.LoggerFactory;
|
||||||
*/
|
*/
|
||||||
public class DeadNodeHandler implements EventHandler<DatanodeDetails> {
|
public class DeadNodeHandler implements EventHandler<DatanodeDetails> {
|
||||||
|
|
||||||
private final ContainerStateManager containerStateManager;
|
private final ContainerManager containerManager;
|
||||||
|
|
||||||
private final NodeManager nodeManager;
|
private final NodeManager nodeManager;
|
||||||
|
|
||||||
|
@ -45,8 +48,8 @@ public class DeadNodeHandler implements EventHandler<DatanodeDetails> {
|
||||||
LoggerFactory.getLogger(DeadNodeHandler.class);
|
LoggerFactory.getLogger(DeadNodeHandler.class);
|
||||||
|
|
||||||
public DeadNodeHandler(NodeManager nodeManager,
|
public DeadNodeHandler(NodeManager nodeManager,
|
||||||
ContainerStateManager containerStateManager) {
|
ContainerManager containerManager) {
|
||||||
this.containerStateManager = containerStateManager;
|
this.containerManager = containerManager;
|
||||||
this.nodeManager = nodeManager;
|
this.nodeManager = nodeManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -55,45 +58,58 @@ public class DeadNodeHandler implements EventHandler<DatanodeDetails> {
|
||||||
EventPublisher publisher) {
|
EventPublisher publisher) {
|
||||||
nodeManager.processDeadNode(datanodeDetails.getUuid());
|
nodeManager.processDeadNode(datanodeDetails.getUuid());
|
||||||
|
|
||||||
Set<ContainerID> containers =
|
// TODO: check if there are any pipeline on this node and fire close
|
||||||
|
// pipeline event
|
||||||
|
Set<ContainerID> ids =
|
||||||
nodeManager.getContainers(datanodeDetails.getUuid());
|
nodeManager.getContainers(datanodeDetails.getUuid());
|
||||||
if (containers == null) {
|
if (ids == null) {
|
||||||
LOG.info("There's no containers in dead datanode {}, no replica will be"
|
LOG.info("There's no containers in dead datanode {}, no replica will be"
|
||||||
+ " removed from the in-memory state.", datanodeDetails.getUuid());
|
+ " removed from the in-memory state.", datanodeDetails.getUuid());
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
LOG.info(
|
LOG.info("Datanode {} is dead. Removing replications from the in-memory" +
|
||||||
"Datanode {} is dead. Removing replications from the in-memory state.",
|
" state.", datanodeDetails.getUuid());
|
||||||
datanodeDetails.getUuid());
|
for (ContainerID id : ids) {
|
||||||
for (ContainerID container : containers) {
|
|
||||||
try {
|
try {
|
||||||
try {
|
final ContainerInfo container = containerManager.getContainer(id);
|
||||||
containerStateManager.removeContainerReplica(container,
|
if (!container.isOpen()) {
|
||||||
datanodeDetails);
|
final ContainerReplica replica = ContainerReplica.newBuilder()
|
||||||
} catch (SCMException ex) {
|
.setContainerID(id)
|
||||||
LOG.info("DataNode {} doesn't have replica for container {}.",
|
.setDatanodeDetails(datanodeDetails)
|
||||||
datanodeDetails.getUuid(), container.getId());
|
.build();
|
||||||
}
|
try {
|
||||||
|
containerManager.removeContainerReplica(id, replica);
|
||||||
if (!containerStateManager.isOpen(container)) {
|
replicateIfNeeded(container, publisher);
|
||||||
ReplicationRequest replicationRequest =
|
} catch (ContainerException ex) {
|
||||||
containerStateManager.checkReplicationState(container);
|
LOG.warn("Exception while removing container replica #{} for " +
|
||||||
|
"container #{}.", replica, container, ex);
|
||||||
if (replicationRequest != null) {
|
|
||||||
publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
|
|
||||||
replicationRequest);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (SCMException e) {
|
} catch (ContainerNotFoundException cnfe) {
|
||||||
LOG.error("Can't remove container from containerStateMap {}", container
|
LOG.warn("Container Not found!", cnfe);
|
||||||
.getId(), e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Compare the existing replication number with the expected one.
|
||||||
|
*/
|
||||||
|
private void replicateIfNeeded(ContainerInfo container,
|
||||||
|
EventPublisher publisher) throws ContainerNotFoundException {
|
||||||
|
final int existingReplicas = containerManager
|
||||||
|
.getContainerReplicas(container.containerID()).size();
|
||||||
|
final int expectedReplicas = container.getReplicationFactor().getNumber();
|
||||||
|
if (existingReplicas != expectedReplicas) {
|
||||||
|
publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
|
||||||
|
new ReplicationRequest(
|
||||||
|
container.getContainerID(), existingReplicas, expectedReplicas));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns logger.
|
* Returns logger.
|
||||||
* */
|
* */
|
||||||
|
// TODO: remove this.
|
||||||
public static Logger getLogger() {
|
public static Logger getLogger() {
|
||||||
return LOG;
|
return LOG;
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hdds.HddsConfigKeys;
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
|
||||||
.NodeRegistrationContainerReport;
|
.NodeRegistrationContainerReport;
|
||||||
|
|
|
@ -33,8 +33,10 @@ import org.apache.hadoop.hdds.protocol.proto
|
||||||
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.ScmUtils;
|
import org.apache.hadoop.hdds.scm.ScmUtils;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
|
||||||
|
@ -169,7 +171,7 @@ public class SCMClientProtocolServer implements
|
||||||
String remoteUser = getRpcRemoteUsername();
|
String remoteUser = getRpcRemoteUsername();
|
||||||
getScm().checkAdminAccess(remoteUser);
|
getScm().checkAdminAccess(remoteUser);
|
||||||
return scm.getContainerManager()
|
return scm.getContainerManager()
|
||||||
.getContainer(containerID);
|
.getContainer(ContainerID.valueof(containerID));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -177,8 +179,8 @@ public class SCMClientProtocolServer implements
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (chillModePrecheck.isInChillMode()) {
|
if (chillModePrecheck.isInChillMode()) {
|
||||||
ContainerInfo contInfo = scm.getContainerManager()
|
ContainerInfo contInfo = scm.getContainerManager()
|
||||||
.getContainer(containerID);
|
.getContainer(ContainerID.valueof(containerID));
|
||||||
if (contInfo.isContainerOpen()) {
|
if (contInfo.isOpen()) {
|
||||||
if (!hasRequiredReplicas(contInfo)) {
|
if (!hasRequiredReplicas(contInfo)) {
|
||||||
throw new SCMException("Open container " + containerID + " doesn't"
|
throw new SCMException("Open container " + containerID + " doesn't"
|
||||||
+ " have enough replicas to service this operation in "
|
+ " have enough replicas to service this operation in "
|
||||||
|
@ -189,7 +191,7 @@ public class SCMClientProtocolServer implements
|
||||||
String remoteUser = getRpcRemoteUsername();
|
String remoteUser = getRpcRemoteUsername();
|
||||||
getScm().checkAdminAccess(null);
|
getScm().checkAdminAccess(null);
|
||||||
return scm.getContainerManager()
|
return scm.getContainerManager()
|
||||||
.getContainerWithPipeline(containerID);
|
.getContainerWithPipeline(ContainerID.valueof(containerID));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -198,10 +200,10 @@ public class SCMClientProtocolServer implements
|
||||||
*/
|
*/
|
||||||
private boolean hasRequiredReplicas(ContainerInfo contInfo) {
|
private boolean hasRequiredReplicas(ContainerInfo contInfo) {
|
||||||
try{
|
try{
|
||||||
return getScm().getContainerManager().getStateManager()
|
return getScm().getContainerManager()
|
||||||
.getContainerReplicas(contInfo.containerID())
|
.getContainerReplicas(contInfo.containerID())
|
||||||
.size() >= contInfo.getReplicationFactor().getNumber();
|
.size() >= contInfo.getReplicationFactor().getNumber();
|
||||||
} catch (SCMException ex) {
|
} catch (ContainerNotFoundException ex) {
|
||||||
// getContainerReplicas throws exception if no replica's exist for given
|
// getContainerReplicas throws exception if no replica's exist for given
|
||||||
// container.
|
// container.
|
||||||
return false;
|
return false;
|
||||||
|
@ -212,14 +214,14 @@ public class SCMClientProtocolServer implements
|
||||||
public List<ContainerInfo> listContainer(long startContainerID,
|
public List<ContainerInfo> listContainer(long startContainerID,
|
||||||
int count) throws IOException {
|
int count) throws IOException {
|
||||||
return scm.getContainerManager().
|
return scm.getContainerManager().
|
||||||
listContainer(startContainerID, count);
|
listContainer(ContainerID.valueof(startContainerID), count);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void deleteContainer(long containerID) throws IOException {
|
public void deleteContainer(long containerID) throws IOException {
|
||||||
String remoteUser = getRpcRemoteUsername();
|
String remoteUser = getRpcRemoteUsername();
|
||||||
getScm().checkAdminAccess(remoteUser);
|
getScm().checkAdminAccess(remoteUser);
|
||||||
scm.getContainerManager().deleteContainer(containerID);
|
scm.getContainerManager().deleteContainer(ContainerID.valueof(containerID));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -257,10 +259,12 @@ public class SCMClientProtocolServer implements
|
||||||
.ObjectStageChangeRequestProto.Op.create) {
|
.ObjectStageChangeRequestProto.Op.create) {
|
||||||
if (stage == StorageContainerLocationProtocolProtos
|
if (stage == StorageContainerLocationProtocolProtos
|
||||||
.ObjectStageChangeRequestProto.Stage.begin) {
|
.ObjectStageChangeRequestProto.Stage.begin) {
|
||||||
scm.getContainerManager().updateContainerState(id, HddsProtos
|
scm.getContainerManager().updateContainerState(
|
||||||
|
ContainerID.valueof(id), HddsProtos
|
||||||
.LifeCycleEvent.CREATE);
|
.LifeCycleEvent.CREATE);
|
||||||
} else {
|
} else {
|
||||||
scm.getContainerManager().updateContainerState(id, HddsProtos
|
scm.getContainerManager().updateContainerState(
|
||||||
|
ContainerID.valueof(id), HddsProtos
|
||||||
.LifeCycleEvent.CREATED);
|
.LifeCycleEvent.CREATED);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
@ -268,10 +272,12 @@ public class SCMClientProtocolServer implements
|
||||||
.ObjectStageChangeRequestProto.Op.close) {
|
.ObjectStageChangeRequestProto.Op.close) {
|
||||||
if (stage == StorageContainerLocationProtocolProtos
|
if (stage == StorageContainerLocationProtocolProtos
|
||||||
.ObjectStageChangeRequestProto.Stage.begin) {
|
.ObjectStageChangeRequestProto.Stage.begin) {
|
||||||
scm.getContainerManager().updateContainerState(id, HddsProtos
|
scm.getContainerManager().updateContainerState(
|
||||||
|
ContainerID.valueof(id), HddsProtos
|
||||||
.LifeCycleEvent.FINALIZE);
|
.LifeCycleEvent.FINALIZE);
|
||||||
} else {
|
} else {
|
||||||
scm.getContainerManager().updateContainerState(id, HddsProtos
|
scm.getContainerManager().updateContainerState(
|
||||||
|
ContainerID.valueof(id), HddsProtos
|
||||||
.LifeCycleEvent.CLOSE);
|
.LifeCycleEvent.CLOSE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -197,7 +197,7 @@ public class SCMDatanodeProtocolServer implements
|
||||||
if (registeredCommand.getError()
|
if (registeredCommand.getError()
|
||||||
== SCMRegisteredResponseProto.ErrorCode.success) {
|
== SCMRegisteredResponseProto.ErrorCode.success) {
|
||||||
scm.getContainerManager().processContainerReports(datanodeDetails,
|
scm.getContainerManager().processContainerReports(datanodeDetails,
|
||||||
containerReportsProto, true);
|
containerReportsProto);
|
||||||
eventPublisher.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
eventPublisher.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
||||||
new NodeRegistrationContainerReport(datanodeDetails,
|
new NodeRegistrationContainerReport(datanodeDetails,
|
||||||
containerReportsProto));
|
containerReportsProto));
|
||||||
|
|
|
@ -40,13 +40,14 @@ import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
|
||||||
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
|
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
|
||||||
import org.apache.hadoop.hdds.scm.container.CloseContainerWatcher;
|
import org.apache.hadoop.hdds.scm.container.CloseContainerWatcher;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerActionsHandler;
|
import org.apache.hadoop.hdds.scm.container.ContainerActionsHandler;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerReportHandler;
|
import org.apache.hadoop.hdds.scm.container.ContainerReportHandler;
|
||||||
import org.apache.hadoop.hdds.scm.container.replication
|
import org.apache.hadoop.hdds.scm.container.replication
|
||||||
.ReplicationActivityStatus;
|
.ReplicationActivityStatus;
|
||||||
import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
|
import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
||||||
.ContainerPlacementPolicy;
|
.ContainerPlacementPolicy;
|
||||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
||||||
|
@ -97,9 +98,6 @@ import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_DEFAULT;
|
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_MB;
|
|
||||||
|
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
|
||||||
import static org.apache.hadoop.util.ExitUtil.terminate;
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
||||||
|
|
||||||
|
@ -190,9 +188,6 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
*/
|
*/
|
||||||
private StorageContainerManager(OzoneConfiguration conf) throws IOException {
|
private StorageContainerManager(OzoneConfiguration conf) throws IOException {
|
||||||
|
|
||||||
final int cacheSize = conf.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
|
|
||||||
OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
|
|
||||||
|
|
||||||
StorageContainerManager.initMetrics();
|
StorageContainerManager.initMetrics();
|
||||||
initContainerReportCache(conf);
|
initContainerReportCache(conf);
|
||||||
|
|
||||||
|
@ -207,9 +202,9 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
scmNodeManager = new SCMNodeManager(
|
scmNodeManager = new SCMNodeManager(
|
||||||
conf, scmStorage.getClusterID(), this, eventQueue);
|
conf, scmStorage.getClusterID(), this, eventQueue);
|
||||||
containerManager = new SCMContainerManager(
|
containerManager = new SCMContainerManager(
|
||||||
conf, getScmNodeManager(), cacheSize, eventQueue);
|
conf, scmNodeManager, eventQueue);
|
||||||
scmBlockManager = new BlockManagerImpl(
|
scmBlockManager = new BlockManagerImpl(
|
||||||
conf, getScmNodeManager(), containerManager, eventQueue);
|
conf, scmNodeManager, containerManager, eventQueue);
|
||||||
|
|
||||||
replicationStatus = new ReplicationActivityStatus();
|
replicationStatus = new ReplicationActivityStatus();
|
||||||
|
|
||||||
|
@ -227,7 +222,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
StaleNodeHandler staleNodeHandler =
|
StaleNodeHandler staleNodeHandler =
|
||||||
new StaleNodeHandler(containerManager.getPipelineSelector());
|
new StaleNodeHandler(containerManager.getPipelineSelector());
|
||||||
DeadNodeHandler deadNodeHandler = new DeadNodeHandler(scmNodeManager,
|
DeadNodeHandler deadNodeHandler = new DeadNodeHandler(scmNodeManager,
|
||||||
getContainerManager().getStateManager());
|
containerManager);
|
||||||
ContainerActionsHandler actionsHandler = new ContainerActionsHandler();
|
ContainerActionsHandler actionsHandler = new ContainerActionsHandler();
|
||||||
PendingDeleteHandler pendingDeleteHandler =
|
PendingDeleteHandler pendingDeleteHandler =
|
||||||
new PendingDeleteHandler(scmBlockManager.getSCMBlockDeletingService());
|
new PendingDeleteHandler(scmBlockManager.getSCMBlockDeletingService());
|
||||||
|
@ -236,7 +231,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
new ContainerReportHandler(containerManager, scmNodeManager,
|
new ContainerReportHandler(containerManager, scmNodeManager,
|
||||||
replicationStatus);
|
replicationStatus);
|
||||||
scmChillModeManager = new SCMChillModeManager(conf,
|
scmChillModeManager = new SCMChillModeManager(conf,
|
||||||
getContainerManager().getStateManager().getAllContainers(),
|
containerManager.getContainers(),
|
||||||
eventQueue);
|
eventQueue);
|
||||||
PipelineActionEventHandler pipelineActionEventHandler =
|
PipelineActionEventHandler pipelineActionEventHandler =
|
||||||
new PipelineActionEventHandler();
|
new PipelineActionEventHandler();
|
||||||
|
@ -263,8 +258,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
new SCMContainerPlacementCapacity(scmNodeManager, conf);
|
new SCMContainerPlacementCapacity(scmNodeManager, conf);
|
||||||
|
|
||||||
replicationManager = new ReplicationManager(containerPlacementPolicy,
|
replicationManager = new ReplicationManager(containerPlacementPolicy,
|
||||||
containerManager.getStateManager(), eventQueue,
|
containerManager, eventQueue, commandWatcherLeaseManager);
|
||||||
commandWatcherLeaseManager);
|
|
||||||
|
|
||||||
// setup CloseContainer watcher
|
// setup CloseContainer watcher
|
||||||
CloseContainerWatcher closeContainerWatcher =
|
CloseContainerWatcher closeContainerWatcher =
|
||||||
|
@ -632,7 +626,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public ContainerInfo getContainerInfo(long containerID) throws
|
public ContainerInfo getContainerInfo(long containerID) throws
|
||||||
IOException {
|
IOException {
|
||||||
return containerManager.getContainer(containerID);
|
return containerManager.getContainer(ContainerID.valueof(containerID));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -22,7 +22,7 @@ import java.util.List;
|
||||||
import org.apache.commons.lang3.RandomUtils;
|
import org.apache.commons.lang3.RandomUtils;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos;
|
.StorageContainerDatanodeProtocolProtos;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
|
||||||
.NodeRegistrationContainerReport;
|
.NodeRegistrationContainerReport;
|
||||||
|
|
||||||
|
|
|
@ -23,11 +23,10 @@ import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
|
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
|
||||||
import org.apache.hadoop.hdds.scm.server
|
import org.apache.hadoop.hdds.scm.server
|
||||||
.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
|
.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
|
||||||
import org.mockito.Mockito;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import static org.mockito.Mockito.when;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.ContainerInfo;
|
.StorageContainerDatanodeProtocolProtos.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
|
@ -42,13 +41,8 @@ import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
|
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.StorageTypeProto;
|
.StorageContainerDatanodeProtocolProtos.StorageTypeProto;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
||||||
import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
|
import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
|
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
|
||||||
|
|
||||||
|
@ -412,39 +406,21 @@ public final class TestUtils {
|
||||||
return report.build();
|
return report.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static
|
public static org.apache.hadoop.hdds.scm.container.ContainerInfo
|
||||||
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo
|
allocateContainer(ContainerManager containerManager)
|
||||||
allocateContainer(ContainerStateManager containerStateManager)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
return containerManager
|
||||||
PipelineSelector pipelineSelector = Mockito.mock(PipelineSelector.class);
|
.allocateContainer(HddsProtos.ReplicationType.STAND_ALONE,
|
||||||
|
|
||||||
Pipeline pipeline = new Pipeline("leader", HddsProtos.LifeCycleState.CLOSED,
|
|
||||||
HddsProtos.ReplicationType.STAND_ALONE,
|
|
||||||
HddsProtos.ReplicationFactor.THREE,
|
|
||||||
PipelineID.randomId());
|
|
||||||
|
|
||||||
when(pipelineSelector
|
|
||||||
.getReplicationPipeline(HddsProtos.ReplicationType.STAND_ALONE,
|
|
||||||
HddsProtos.ReplicationFactor.THREE)).thenReturn(pipeline);
|
|
||||||
|
|
||||||
return containerStateManager
|
|
||||||
.allocateContainer(pipelineSelector,
|
|
||||||
HddsProtos.ReplicationType.STAND_ALONE,
|
|
||||||
HddsProtos.ReplicationFactor.THREE, "root").getContainerInfo();
|
HddsProtos.ReplicationFactor.THREE, "root").getContainerInfo();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void closeContainer(ContainerStateManager containerStateManager,
|
public static void closeContainer(ContainerManager containerManager,
|
||||||
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo
|
ContainerID id) throws IOException {
|
||||||
container)
|
containerManager.updateContainerState(
|
||||||
throws SCMException {
|
id, HddsProtos.LifeCycleEvent.FINALIZE);
|
||||||
|
containerManager.updateContainerState(
|
||||||
containerStateManager.getContainerStateMap()
|
id, HddsProtos.LifeCycleEvent.CLOSE);
|
||||||
.updateState(container, container.getState(), LifeCycleState.CLOSING);
|
|
||||||
|
|
||||||
containerStateManager.getContainerStateMap()
|
|
||||||
.updateState(container, container.getState(), LifeCycleState.CLOSED);
|
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -83,7 +83,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
throw new IOException("Unable to create test directory path");
|
throw new IOException("Unable to create test directory path");
|
||||||
}
|
}
|
||||||
nodeManager = new MockNodeManager(true, 10);
|
nodeManager = new MockNodeManager(true, 10);
|
||||||
mapping = new SCMContainerManager(conf, nodeManager, 128, eventQueue);
|
mapping = new SCMContainerManager(conf, nodeManager, eventQueue);
|
||||||
blockManager = new BlockManagerImpl(conf,
|
blockManager = new BlockManagerImpl(conf,
|
||||||
nodeManager, mapping, eventQueue);
|
nodeManager, mapping, eventQueue);
|
||||||
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);
|
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);
|
||||||
|
|
|
@ -19,9 +19,10 @@ package org.apache.hadoop.hdds.scm.block;
|
||||||
|
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.commons.lang3.RandomUtils;
|
import org.apache.commons.lang3.RandomUtils;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
||||||
|
@ -61,7 +62,7 @@ import java.util.stream.Collectors;
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||||
.OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
|
.OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
|
||||||
import static org.mockito.Matchers.anyLong;
|
import static org.mockito.Matchers.anyObject;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -109,9 +110,10 @@ public class TestDeletedBlockLog {
|
||||||
pipeline.addMember(dnList.get(2));
|
pipeline.addMember(dnList.get(2));
|
||||||
ContainerWithPipeline containerWithPipeline =
|
ContainerWithPipeline containerWithPipeline =
|
||||||
new ContainerWithPipeline(containerInfo, pipeline);
|
new ContainerWithPipeline(containerInfo, pipeline);
|
||||||
when(containerManager.getContainerWithPipeline(anyLong()))
|
when(containerManager.getContainerWithPipeline(anyObject()))
|
||||||
.thenReturn(containerWithPipeline);
|
.thenReturn(containerWithPipeline);
|
||||||
when(containerManager.getContainer(anyLong())).thenReturn(containerInfo);
|
when(containerManager.getContainer(anyObject()))
|
||||||
|
.thenReturn(containerInfo);
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -396,8 +398,8 @@ public class TestDeletedBlockLog {
|
||||||
ContainerWithPipeline containerWithPipeline = new ContainerWithPipeline(
|
ContainerWithPipeline containerWithPipeline = new ContainerWithPipeline(
|
||||||
containerInfo, pipeline);
|
containerInfo, pipeline);
|
||||||
Mockito.doReturn(containerInfo).when(containerManager)
|
Mockito.doReturn(containerInfo).when(containerManager)
|
||||||
.getContainer(containerID);
|
.getContainer(ContainerID.valueof(containerID));
|
||||||
Mockito.doReturn(containerWithPipeline).when(containerManager)
|
Mockito.doReturn(containerWithPipeline).when(containerManager)
|
||||||
.getContainerWithPipeline(containerID);
|
.getContainerWithPipeline(ContainerID.valueof(containerID));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -50,7 +50,7 @@ public class TestCloseContainerEventHandler {
|
||||||
|
|
||||||
private static Configuration configuration;
|
private static Configuration configuration;
|
||||||
private static MockNodeManager nodeManager;
|
private static MockNodeManager nodeManager;
|
||||||
private static SCMContainerManager mapping;
|
private static SCMContainerManager containerManager;
|
||||||
private static long size;
|
private static long size;
|
||||||
private static File testDir;
|
private static File testDir;
|
||||||
private static EventQueue eventQueue;
|
private static EventQueue eventQueue;
|
||||||
|
@ -65,18 +65,18 @@ public class TestCloseContainerEventHandler {
|
||||||
configuration
|
configuration
|
||||||
.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
|
.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
|
||||||
nodeManager = new MockNodeManager(true, 10);
|
nodeManager = new MockNodeManager(true, 10);
|
||||||
mapping = new SCMContainerManager(configuration, nodeManager, 128,
|
containerManager = new SCMContainerManager(configuration, nodeManager,
|
||||||
new EventQueue());
|
new EventQueue());
|
||||||
eventQueue = new EventQueue();
|
eventQueue = new EventQueue();
|
||||||
eventQueue.addHandler(CLOSE_CONTAINER,
|
eventQueue.addHandler(CLOSE_CONTAINER,
|
||||||
new CloseContainerEventHandler(mapping));
|
new CloseContainerEventHandler(containerManager));
|
||||||
eventQueue.addHandler(DATANODE_COMMAND, nodeManager);
|
eventQueue.addHandler(DATANODE_COMMAND, nodeManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterClass
|
@AfterClass
|
||||||
public static void tearDown() throws Exception {
|
public static void tearDown() throws Exception {
|
||||||
if (mapping != null) {
|
if (containerManager != null) {
|
||||||
mapping.close();
|
containerManager.close();
|
||||||
}
|
}
|
||||||
FileUtil.fullyDelete(testDir);
|
FileUtil.fullyDelete(testDir);
|
||||||
}
|
}
|
||||||
|
@ -109,7 +109,7 @@ public class TestCloseContainerEventHandler {
|
||||||
|
|
||||||
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
||||||
.captureLogs(CloseContainerEventHandler.LOG);
|
.captureLogs(CloseContainerEventHandler.LOG);
|
||||||
ContainerWithPipeline containerWithPipeline = mapping
|
ContainerWithPipeline containerWithPipeline = containerManager
|
||||||
.allocateContainer(HddsProtos.ReplicationType.STAND_ALONE,
|
.allocateContainer(HddsProtos.ReplicationType.STAND_ALONE,
|
||||||
HddsProtos.ReplicationFactor.ONE, "ozone");
|
HddsProtos.ReplicationFactor.ONE, "ozone");
|
||||||
ContainerID id = new ContainerID(
|
ContainerID id = new ContainerID(
|
||||||
|
@ -123,7 +123,7 @@ public class TestCloseContainerEventHandler {
|
||||||
// command in the Datanode
|
// command in the Datanode
|
||||||
Assert.assertEquals(0, nodeManager.getCommandCount(datanode));
|
Assert.assertEquals(0, nodeManager.getCommandCount(datanode));
|
||||||
//Execute these state transitions so that we can close the container.
|
//Execute these state transitions so that we can close the container.
|
||||||
mapping.updateContainerState(id.getId(), CREATED);
|
containerManager.updateContainerState(id, CREATED);
|
||||||
eventQueue.fireEvent(CLOSE_CONTAINER,
|
eventQueue.fireEvent(CLOSE_CONTAINER,
|
||||||
new ContainerID(
|
new ContainerID(
|
||||||
containerWithPipeline.getContainerInfo().getContainerID()));
|
containerWithPipeline.getContainerInfo().getContainerID()));
|
||||||
|
@ -131,7 +131,7 @@ public class TestCloseContainerEventHandler {
|
||||||
Assert.assertEquals(closeCount + 1,
|
Assert.assertEquals(closeCount + 1,
|
||||||
nodeManager.getCommandCount(datanode));
|
nodeManager.getCommandCount(datanode));
|
||||||
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
|
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
|
||||||
mapping.getStateManager().getContainer(id).getState());
|
containerManager.getContainer(id).getState());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -139,7 +139,7 @@ public class TestCloseContainerEventHandler {
|
||||||
|
|
||||||
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
||||||
.captureLogs(CloseContainerEventHandler.LOG);
|
.captureLogs(CloseContainerEventHandler.LOG);
|
||||||
ContainerWithPipeline containerWithPipeline = mapping
|
ContainerWithPipeline containerWithPipeline = containerManager
|
||||||
.allocateContainer(HddsProtos.ReplicationType.RATIS,
|
.allocateContainer(HddsProtos.ReplicationType.RATIS,
|
||||||
HddsProtos.ReplicationFactor.THREE, "ozone");
|
HddsProtos.ReplicationFactor.THREE, "ozone");
|
||||||
ContainerID id = new ContainerID(
|
ContainerID id = new ContainerID(
|
||||||
|
@ -160,7 +160,7 @@ public class TestCloseContainerEventHandler {
|
||||||
i++;
|
i++;
|
||||||
}
|
}
|
||||||
//Execute these state transitions so that we can close the container.
|
//Execute these state transitions so that we can close the container.
|
||||||
mapping.updateContainerState(id.getId(), CREATED);
|
containerManager.updateContainerState(id, CREATED);
|
||||||
eventQueue.fireEvent(CLOSE_CONTAINER, id);
|
eventQueue.fireEvent(CLOSE_CONTAINER, id);
|
||||||
eventQueue.processAll(1000);
|
eventQueue.processAll(1000);
|
||||||
i = 0;
|
i = 0;
|
||||||
|
@ -170,7 +170,7 @@ public class TestCloseContainerEventHandler {
|
||||||
Assert.assertEquals(closeCount[i] + 1,
|
Assert.assertEquals(closeCount[i] + 1,
|
||||||
nodeManager.getCommandCount(details));
|
nodeManager.getCommandCount(details));
|
||||||
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
|
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
|
||||||
mapping.getStateManager().getContainer(id).getState());
|
containerManager.getContainer(id).getState());
|
||||||
i++;
|
i++;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,34 +23,28 @@ import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
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.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
||||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo
|
|
||||||
.Builder;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.replication
|
import org.apache.hadoop.hdds.scm.container.replication
|
||||||
.ReplicationActivityStatus;
|
.ReplicationActivityStatus;
|
||||||
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
||||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||||
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
|
||||||
.ContainerReportFromDatanode;
|
.ContainerReportFromDatanode;
|
||||||
import org.apache.hadoop.hdds.server.events.Event;
|
import org.apache.hadoop.hdds.server.events.Event;
|
||||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||||
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
import org.junit.Ignore;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import static org.mockito.Matchers.anyLong;
|
|
||||||
import org.mockito.Mockito;
|
|
||||||
import static org.mockito.Mockito.when;
|
|
||||||
import org.mockito.stubbing.Answer;
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -60,7 +54,7 @@ import org.slf4j.LoggerFactory;
|
||||||
public class TestContainerReportHandler implements EventPublisher {
|
public class TestContainerReportHandler implements EventPublisher {
|
||||||
|
|
||||||
private List<Object> publishedEvents = new ArrayList<>();
|
private List<Object> publishedEvents = new ArrayList<>();
|
||||||
private final NodeManager nodeManager = new MockNodeManager(true, 1);
|
private final NodeManager nodeManager = new MockNodeManager(true, 15);
|
||||||
|
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(TestContainerReportHandler.class);
|
LoggerFactory.getLogger(TestContainerReportHandler.class);
|
||||||
|
@ -70,27 +64,17 @@ public class TestContainerReportHandler implements EventPublisher {
|
||||||
publishedEvents.clear();
|
publishedEvents.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
//TODO: Rewrite it
|
||||||
|
@Ignore
|
||||||
@Test
|
@Test
|
||||||
public void test() throws IOException {
|
public void test() throws IOException {
|
||||||
|
String testDir = GenericTestUtils.getTempPath(
|
||||||
|
this.getClass().getSimpleName());
|
||||||
//GIVEN
|
//GIVEN
|
||||||
OzoneConfiguration conf = new OzoneConfiguration();
|
OzoneConfiguration conf = new OzoneConfiguration();
|
||||||
ContainerManager containerManager = Mockito.mock(ContainerManager.class);
|
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir);
|
||||||
PipelineSelector selector = Mockito.mock(PipelineSelector.class);
|
SCMContainerManager containerManager = new SCMContainerManager(
|
||||||
|
conf, nodeManager, new EventQueue());
|
||||||
when(containerManager.getContainer(anyLong()))
|
|
||||||
.thenAnswer(
|
|
||||||
(Answer<ContainerInfo>) invocation ->
|
|
||||||
new Builder()
|
|
||||||
.setReplicationFactor(ReplicationFactor.THREE)
|
|
||||||
.setContainerID((Long) invocation.getArguments()[0])
|
|
||||||
.setState(LifeCycleState.CLOSED)
|
|
||||||
.build()
|
|
||||||
);
|
|
||||||
|
|
||||||
ContainerStateManager containerStateManager =
|
|
||||||
new ContainerStateManager(conf, containerManager, selector);
|
|
||||||
|
|
||||||
when(containerManager.getStateManager()).thenReturn(containerStateManager);
|
|
||||||
|
|
||||||
ReplicationActivityStatus replicationActivityStatus =
|
ReplicationActivityStatus replicationActivityStatus =
|
||||||
new ReplicationActivityStatus();
|
new ReplicationActivityStatus();
|
||||||
|
@ -107,24 +91,16 @@ public class TestContainerReportHandler implements EventPublisher {
|
||||||
nodeManager.addDatanodeInContainerMap(dn2.getUuid(), new HashSet<>());
|
nodeManager.addDatanodeInContainerMap(dn2.getUuid(), new HashSet<>());
|
||||||
nodeManager.addDatanodeInContainerMap(dn3.getUuid(), new HashSet<>());
|
nodeManager.addDatanodeInContainerMap(dn3.getUuid(), new HashSet<>());
|
||||||
nodeManager.addDatanodeInContainerMap(dn4.getUuid(), new HashSet<>());
|
nodeManager.addDatanodeInContainerMap(dn4.getUuid(), new HashSet<>());
|
||||||
PipelineSelector pipelineSelector = Mockito.mock(PipelineSelector.class);
|
|
||||||
|
|
||||||
Pipeline pipeline = new Pipeline("leader", LifeCycleState.CLOSED,
|
ContainerInfo cont1 = containerManager
|
||||||
ReplicationType.STAND_ALONE, ReplicationFactor.THREE,
|
.allocateContainer(ReplicationType.STAND_ALONE,
|
||||||
PipelineID.randomId());
|
|
||||||
|
|
||||||
when(pipelineSelector.getReplicationPipeline(ReplicationType.STAND_ALONE,
|
|
||||||
ReplicationFactor.THREE)).thenReturn(pipeline);
|
|
||||||
|
|
||||||
ContainerInfo cont1 = containerStateManager
|
|
||||||
.allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
|
|
||||||
ReplicationFactor.THREE, "root").getContainerInfo();
|
ReplicationFactor.THREE, "root").getContainerInfo();
|
||||||
ContainerInfo cont2 = containerStateManager
|
ContainerInfo cont2 = containerManager
|
||||||
.allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
|
.allocateContainer(ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.THREE, "root").getContainerInfo();
|
ReplicationFactor.THREE, "root").getContainerInfo();
|
||||||
// Open Container
|
// Open Container
|
||||||
ContainerInfo cont3 = containerStateManager
|
ContainerInfo cont3 = containerManager
|
||||||
.allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
|
.allocateContainer(ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.THREE, "root").getContainerInfo();
|
ReplicationFactor.THREE, "root").getContainerInfo();
|
||||||
|
|
||||||
long c1 = cont1.getContainerID();
|
long c1 = cont1.getContainerID();
|
||||||
|
@ -132,8 +108,8 @@ public class TestContainerReportHandler implements EventPublisher {
|
||||||
long c3 = cont3.getContainerID();
|
long c3 = cont3.getContainerID();
|
||||||
|
|
||||||
// Close remaining containers
|
// Close remaining containers
|
||||||
TestUtils.closeContainer(containerStateManager, cont1);
|
TestUtils.closeContainer(containerManager, cont1.containerID());
|
||||||
TestUtils.closeContainer(containerStateManager, cont2);
|
TestUtils.closeContainer(containerManager, cont2.containerID());
|
||||||
|
|
||||||
//when
|
//when
|
||||||
|
|
||||||
|
|
|
@ -18,19 +18,23 @@
|
||||||
package org.apache.hadoop.hdds.scm.container;
|
package org.apache.hadoop.hdds.scm.container;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
||||||
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Testing ContainerStatemanager.
|
* Testing ContainerStatemanager.
|
||||||
*/
|
*/
|
||||||
|
@ -41,16 +45,14 @@ public class TestContainerStateManager {
|
||||||
@Before
|
@Before
|
||||||
public void init() throws IOException {
|
public void init() throws IOException {
|
||||||
OzoneConfiguration conf = new OzoneConfiguration();
|
OzoneConfiguration conf = new OzoneConfiguration();
|
||||||
ContainerManager mapping = Mockito.mock(ContainerManager.class);
|
containerStateManager = new ContainerStateManager(conf);
|
||||||
PipelineSelector selector = Mockito.mock(PipelineSelector.class);
|
|
||||||
containerStateManager = new ContainerStateManager(conf, mapping, selector);
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void checkReplicationStateOK() throws IOException {
|
public void checkReplicationStateOK() throws IOException {
|
||||||
//GIVEN
|
//GIVEN
|
||||||
ContainerInfo c1 = TestUtils.allocateContainer(containerStateManager);
|
ContainerInfo c1 = allocateContainer();
|
||||||
|
|
||||||
DatanodeDetails d1 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails d1 = TestUtils.randomDatanodeDetails();
|
||||||
DatanodeDetails d2 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails d2 = TestUtils.randomDatanodeDetails();
|
||||||
|
@ -61,18 +63,18 @@ public class TestContainerStateManager {
|
||||||
addReplica(c1, d3);
|
addReplica(c1, d3);
|
||||||
|
|
||||||
//WHEN
|
//WHEN
|
||||||
ReplicationRequest replicationRequest = containerStateManager
|
Set<ContainerReplica> replicas = containerStateManager
|
||||||
.checkReplicationState(new ContainerID(c1.getContainerID()));
|
.getContainerReplicas(c1.containerID());
|
||||||
|
|
||||||
//THEN
|
//THEN
|
||||||
Assert.assertNull(replicationRequest);
|
Assert.assertEquals(3, replicas.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void checkReplicationStateMissingReplica() throws IOException {
|
public void checkReplicationStateMissingReplica() throws IOException {
|
||||||
//GIVEN
|
//GIVEN
|
||||||
|
|
||||||
ContainerInfo c1 = TestUtils.allocateContainer(containerStateManager);
|
ContainerInfo c1 = allocateContainer();
|
||||||
|
|
||||||
DatanodeDetails d1 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails d1 = TestUtils.randomDatanodeDetails();
|
||||||
DatanodeDetails d2 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails d2 = TestUtils.randomDatanodeDetails();
|
||||||
|
@ -81,18 +83,40 @@ public class TestContainerStateManager {
|
||||||
addReplica(c1, d2);
|
addReplica(c1, d2);
|
||||||
|
|
||||||
//WHEN
|
//WHEN
|
||||||
ReplicationRequest replicationRequest = containerStateManager
|
Set<ContainerReplica> replicas = containerStateManager
|
||||||
.checkReplicationState(new ContainerID(c1.getContainerID()));
|
.getContainerReplicas(c1.containerID());
|
||||||
|
|
||||||
Assert
|
Assert.assertEquals(2, replicas.size());
|
||||||
.assertEquals(c1.getContainerID(), replicationRequest.getContainerId());
|
Assert.assertEquals(3, c1.getReplicationFactor().getNumber());
|
||||||
Assert.assertEquals(2, replicationRequest.getReplicationCount());
|
|
||||||
Assert.assertEquals(3, replicationRequest.getExpecReplicationCount());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void addReplica(ContainerInfo c1, DatanodeDetails d1) {
|
private void addReplica(ContainerInfo cont, DatanodeDetails node)
|
||||||
|
throws ContainerNotFoundException {
|
||||||
|
ContainerReplica replica = ContainerReplica.newBuilder()
|
||||||
|
.setContainerID(cont.containerID())
|
||||||
|
.setDatanodeDetails(node)
|
||||||
|
.build();
|
||||||
containerStateManager
|
containerStateManager
|
||||||
.addContainerReplica(new ContainerID(c1.getContainerID()), d1);
|
.updateContainerReplica(cont.containerID(), replica);
|
||||||
|
}
|
||||||
|
|
||||||
|
private ContainerInfo allocateContainer() throws IOException {
|
||||||
|
|
||||||
|
PipelineSelector pipelineSelector = Mockito.mock(PipelineSelector.class);
|
||||||
|
|
||||||
|
Pipeline pipeline = new Pipeline("leader", HddsProtos.LifeCycleState.CLOSED,
|
||||||
|
HddsProtos.ReplicationType.STAND_ALONE,
|
||||||
|
HddsProtos.ReplicationFactor.THREE,
|
||||||
|
PipelineID.randomId());
|
||||||
|
|
||||||
|
when(pipelineSelector
|
||||||
|
.getReplicationPipeline(HddsProtos.ReplicationType.STAND_ALONE,
|
||||||
|
HddsProtos.ReplicationFactor.THREE)).thenReturn(pipeline);
|
||||||
|
|
||||||
|
return containerStateManager.allocateContainer(
|
||||||
|
pipelineSelector, HddsProtos.ReplicationType.STAND_ALONE,
|
||||||
|
HddsProtos.ReplicationFactor.THREE, "root");
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -23,7 +23,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
@ -32,12 +31,10 @@ import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos;
|
.StorageContainerDatanodeProtocolProtos;
|
||||||
import org.apache.hadoop.hdds.protocol.proto
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
||||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
|
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.test.LambdaTestUtils;
|
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -50,7 +47,6 @@ import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.NavigableSet;
|
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.TreeSet;
|
import java.util.TreeSet;
|
||||||
|
@ -89,7 +85,7 @@ public class TestSCMContainerManager {
|
||||||
throw new IOException("Unable to create test directory path");
|
throw new IOException("Unable to create test directory path");
|
||||||
}
|
}
|
||||||
nodeManager = new MockNodeManager(true, 10);
|
nodeManager = new MockNodeManager(true, 10);
|
||||||
containerManager = new SCMContainerManager(conf, nodeManager, 128,
|
containerManager = new SCMContainerManager(conf, nodeManager,
|
||||||
new EventQueue());
|
new EventQueue());
|
||||||
xceiverClientManager = new XceiverClientManager(conf);
|
xceiverClientManager = new XceiverClientManager(conf);
|
||||||
random = new Random();
|
random = new Random();
|
||||||
|
@ -169,28 +165,36 @@ public class TestSCMContainerManager {
|
||||||
.setIpAddress("2.2.2.2")
|
.setIpAddress("2.2.2.2")
|
||||||
.setUuid(UUID.randomUUID().toString()).build();
|
.setUuid(UUID.randomUUID().toString()).build();
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(contInfo.getContainerID(), LifeCycleEvent.CREATE);
|
.updateContainerState(contInfo.containerID(), LifeCycleEvent.CREATE);
|
||||||
containerManager.updateContainerState(contInfo.getContainerID(),
|
containerManager.updateContainerState(contInfo.containerID(),
|
||||||
LifeCycleEvent.CREATED);
|
LifeCycleEvent.CREATED);
|
||||||
containerManager.updateContainerState(contInfo.getContainerID(),
|
containerManager.updateContainerState(contInfo.containerID(),
|
||||||
LifeCycleEvent.FINALIZE);
|
LifeCycleEvent.FINALIZE);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(contInfo.getContainerID(), LifeCycleEvent.CLOSE);
|
.updateContainerState(contInfo.containerID(), LifeCycleEvent.CLOSE);
|
||||||
ContainerInfo finalContInfo = contInfo;
|
ContainerInfo finalContInfo = contInfo;
|
||||||
LambdaTestUtils.intercept(SCMException.class, "No entry exist for "
|
Assert.assertEquals(0,
|
||||||
+ "containerId:", () -> containerManager.getContainerWithPipeline(
|
containerManager.getContainerReplicas(
|
||||||
finalContInfo.getContainerID()));
|
finalContInfo.containerID()).size());
|
||||||
|
|
||||||
containerManager.getStateManager().getContainerStateMap()
|
containerManager.updateContainerReplica(contInfo.containerID(),
|
||||||
.addContainerReplica(contInfo.containerID(), dn1, dn2);
|
ContainerReplica.newBuilder().setContainerID(contInfo.containerID())
|
||||||
|
.setDatanodeDetails(dn1).build());
|
||||||
|
containerManager.updateContainerReplica(contInfo.containerID(),
|
||||||
|
ContainerReplica.newBuilder().setContainerID(contInfo.containerID())
|
||||||
|
.setDatanodeDetails(dn2).build());
|
||||||
|
|
||||||
contInfo = containerManager.getContainer(contInfo.getContainerID());
|
Assert.assertEquals(2,
|
||||||
|
containerManager.getContainerReplicas(
|
||||||
|
finalContInfo.containerID()).size());
|
||||||
|
|
||||||
|
contInfo = containerManager.getContainer(contInfo.containerID());
|
||||||
Assert.assertEquals(contInfo.getState(), LifeCycleState.CLOSED);
|
Assert.assertEquals(contInfo.getState(), LifeCycleState.CLOSED);
|
||||||
Pipeline pipeline = containerWithPipeline.getPipeline();
|
Pipeline pipeline = containerWithPipeline.getPipeline();
|
||||||
containerManager.getPipelineSelector().finalizePipeline(pipeline);
|
containerManager.getPipelineSelector().finalizePipeline(pipeline);
|
||||||
|
|
||||||
ContainerWithPipeline containerWithPipeline2 = containerManager
|
ContainerWithPipeline containerWithPipeline2 = containerManager
|
||||||
.getContainerWithPipeline(contInfo.getContainerID());
|
.getContainerWithPipeline(contInfo.containerID());
|
||||||
pipeline = containerWithPipeline2.getPipeline();
|
pipeline = containerWithPipeline2.getPipeline();
|
||||||
Assert.assertNotEquals(containerWithPipeline, containerWithPipeline2);
|
Assert.assertNotEquals(containerWithPipeline, containerWithPipeline2);
|
||||||
Assert.assertNotNull("Pipeline should not be null", pipeline);
|
Assert.assertNotNull("Pipeline should not be null", pipeline);
|
||||||
|
@ -199,9 +203,14 @@ public class TestSCMContainerManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testgetNoneExistentContainer() throws IOException {
|
public void testgetNoneExistentContainer() {
|
||||||
thrown.expectMessage("Specified key does not exist.");
|
try {
|
||||||
containerManager.getContainer(random.nextLong());
|
containerManager.getContainer(ContainerID.valueof(
|
||||||
|
random.nextInt() & Integer.MAX_VALUE));
|
||||||
|
Assert.fail();
|
||||||
|
} catch (ContainerNotFoundException ex) {
|
||||||
|
// Success!
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -213,21 +222,13 @@ public class TestSCMContainerManager {
|
||||||
xceiverClientManager.getFactor(),
|
xceiverClientManager.getFactor(),
|
||||||
containerOwner);
|
containerOwner);
|
||||||
containerManager.updateContainerState(containerInfo.getContainerInfo()
|
containerManager.updateContainerState(containerInfo.getContainerInfo()
|
||||||
.getContainerID(), HddsProtos.LifeCycleEvent.CREATE);
|
.containerID(), HddsProtos.LifeCycleEvent.CREATE);
|
||||||
Thread.sleep(TIMEOUT + 1000);
|
Thread.sleep(TIMEOUT + 1000);
|
||||||
|
|
||||||
NavigableSet<ContainerID> deleteContainers = containerManager
|
|
||||||
.getStateManager().getMatchingContainerIDs("OZONE",
|
|
||||||
xceiverClientManager.getType(),
|
|
||||||
xceiverClientManager.getFactor(),
|
|
||||||
HddsProtos.LifeCycleState.DELETING);
|
|
||||||
Assert.assertTrue(deleteContainers
|
|
||||||
.contains(containerInfo.getContainerInfo().containerID()));
|
|
||||||
|
|
||||||
thrown.expect(IOException.class);
|
thrown.expect(IOException.class);
|
||||||
thrown.expectMessage("Lease Exception");
|
thrown.expectMessage("Lease Exception");
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(containerInfo.getContainerInfo().getContainerID(),
|
.updateContainerState(containerInfo.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATED);
|
HddsProtos.LifeCycleEvent.CREATED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -257,26 +258,24 @@ public class TestSCMContainerManager {
|
||||||
crBuilder.addAllReports(reports);
|
crBuilder.addAllReports(reports);
|
||||||
|
|
||||||
containerManager.processContainerReports(
|
containerManager.processContainerReports(
|
||||||
datanodeDetails, crBuilder.build(), false);
|
datanodeDetails, crBuilder.build());
|
||||||
|
|
||||||
ContainerInfo updatedContainer =
|
ContainerInfo updatedContainer =
|
||||||
containerManager.getContainer(info.getContainerID());
|
containerManager.getContainer(info.containerID());
|
||||||
Assert.assertEquals(100000000L,
|
Assert.assertEquals(100000000L,
|
||||||
updatedContainer.getNumberOfKeys());
|
updatedContainer.getNumberOfKeys());
|
||||||
Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
|
Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
|
||||||
|
|
||||||
for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
|
for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
|
||||||
LambdaTestUtils.intercept(SCMException.class, "No entry "
|
Assert.assertEquals(containerManager.getContainerReplicas(
|
||||||
+ "exist for containerId:", () -> containerManager.getStateManager()
|
ContainerID.valueof(c.getContainerID())).size(), 1);
|
||||||
.getContainerReplicas(ContainerID.valueof(c.getContainerID())));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
containerManager.processContainerReports(TestUtils.randomDatanodeDetails(),
|
containerManager.processContainerReports(TestUtils.randomDatanodeDetails(),
|
||||||
crBuilder.build(), true);
|
crBuilder.build());
|
||||||
for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
|
for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
|
||||||
Assert.assertTrue(containerManager.getStateManager()
|
Assert.assertEquals(containerManager.getContainerReplicas(
|
||||||
.getContainerReplicas(
|
ContainerID.valueof(c.getContainerID())).size(), 2);
|
||||||
ContainerID.valueof(c.getContainerID())).size() > 0);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -314,9 +313,10 @@ public class TestSCMContainerManager {
|
||||||
crBuilder.addAllReports(reports);
|
crBuilder.addAllReports(reports);
|
||||||
|
|
||||||
containerManager.processContainerReports(
|
containerManager.processContainerReports(
|
||||||
datanodeDetails, crBuilder.build(), false);
|
datanodeDetails, crBuilder.build());
|
||||||
|
|
||||||
List<ContainerInfo> list = containerManager.listContainer(0, 50);
|
List<ContainerInfo> list = containerManager.listContainer(
|
||||||
|
ContainerID.valueof(1), 50);
|
||||||
Assert.assertEquals(2, list.stream().filter(
|
Assert.assertEquals(2, list.stream().filter(
|
||||||
x -> x.getContainerID() == cID1 || x.getContainerID() == cID2).count());
|
x -> x.getContainerID() == cID1 || x.getContainerID() == cID2).count());
|
||||||
Assert.assertEquals(300000000L, list.stream().filter(
|
Assert.assertEquals(300000000L, list.stream().filter(
|
||||||
|
@ -329,23 +329,13 @@ public class TestSCMContainerManager {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCloseContainer() throws IOException {
|
public void testCloseContainer() throws IOException {
|
||||||
ContainerInfo info = createContainer();
|
ContainerID id = createContainer().containerID();
|
||||||
containerManager.updateContainerState(info.getContainerID(),
|
containerManager.updateContainerState(id,
|
||||||
HddsProtos.LifeCycleEvent.FINALIZE);
|
HddsProtos.LifeCycleEvent.FINALIZE);
|
||||||
NavigableSet<ContainerID> pendingCloseContainers = containerManager
|
containerManager.updateContainerState(id,
|
||||||
.getStateManager().getMatchingContainerIDs(containerOwner,
|
|
||||||
xceiverClientManager.getType(),
|
|
||||||
xceiverClientManager.getFactor(),
|
|
||||||
HddsProtos.LifeCycleState.CLOSING);
|
|
||||||
Assert.assertTrue(pendingCloseContainers.contains(info.containerID()));
|
|
||||||
containerManager.updateContainerState(info.getContainerID(),
|
|
||||||
HddsProtos.LifeCycleEvent.CLOSE);
|
HddsProtos.LifeCycleEvent.CLOSE);
|
||||||
NavigableSet<ContainerID> closeContainers = containerManager
|
ContainerInfo closedContainer = containerManager.getContainer(id);
|
||||||
.getStateManager().getMatchingContainerIDs(containerOwner,
|
Assert.assertEquals(LifeCycleState.CLOSED, closedContainer.getState());
|
||||||
xceiverClientManager.getType(),
|
|
||||||
xceiverClientManager.getFactor(),
|
|
||||||
HddsProtos.LifeCycleState.CLOSED);
|
|
||||||
Assert.assertTrue(closeContainers.contains(info.containerID()));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -359,20 +349,11 @@ public class TestSCMContainerManager {
|
||||||
.allocateContainer(xceiverClientManager.getType(),
|
.allocateContainer(xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), containerOwner);
|
xceiverClientManager.getFactor(), containerOwner);
|
||||||
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
|
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
|
||||||
containerManager.updateContainerState(containerInfo.getContainerID(),
|
containerManager.updateContainerState(containerInfo.containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
containerManager.updateContainerState(containerInfo.getContainerID(),
|
containerManager.updateContainerState(containerInfo.containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATED);
|
HddsProtos.LifeCycleEvent.CREATED);
|
||||||
return containerInfo;
|
return containerInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testFlushAllContainers() throws IOException {
|
|
||||||
ContainerInfo info = createContainer();
|
|
||||||
List<ContainerInfo> containers = containerManager.getStateManager()
|
|
||||||
.getAllContainers();
|
|
||||||
Assert.assertTrue(containers.size() > 0);
|
|
||||||
containerManager.flushContainerInfo();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||||
|
@ -31,8 +32,9 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto;
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto;
|
||||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
||||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy;
|
import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy;
|
||||||
|
@ -62,10 +64,11 @@ public class TestReplicationManager {
|
||||||
|
|
||||||
private List<CommandForDatanode<ReplicateContainerCommandProto>> copyEvents;
|
private List<CommandForDatanode<ReplicateContainerCommandProto>> copyEvents;
|
||||||
|
|
||||||
private ContainerStateManager containerStateManager;
|
private ContainerManager containerManager;
|
||||||
|
|
||||||
private ContainerPlacementPolicy containerPlacementPolicy;
|
private ContainerPlacementPolicy containerPlacementPolicy;
|
||||||
private List<DatanodeDetails> listOfDatanodeDetails;
|
private List<DatanodeDetails> listOfDatanodeDetails;
|
||||||
|
private List<ContainerReplica> listOfContainerReplica;
|
||||||
private LeaseManager<Long> leaseManager;
|
private LeaseManager<Long> leaseManager;
|
||||||
private ReplicationManager replicationManager;
|
private ReplicationManager replicationManager;
|
||||||
|
|
||||||
|
@ -73,33 +76,36 @@ public class TestReplicationManager {
|
||||||
public void initReplicationManager() throws IOException {
|
public void initReplicationManager() throws IOException {
|
||||||
|
|
||||||
listOfDatanodeDetails = new ArrayList<>();
|
listOfDatanodeDetails = new ArrayList<>();
|
||||||
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
|
listOfContainerReplica = new ArrayList<>();
|
||||||
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
|
IntStream.range(1, 6).forEach(i -> {
|
||||||
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
|
DatanodeDetails dd = TestUtils.randomDatanodeDetails();
|
||||||
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
|
listOfDatanodeDetails.add(dd);
|
||||||
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
|
listOfContainerReplica.add(ContainerReplica.newBuilder()
|
||||||
|
.setContainerID(ContainerID.valueof(i))
|
||||||
|
.setDatanodeDetails(dd).build());
|
||||||
|
});
|
||||||
|
|
||||||
containerPlacementPolicy =
|
containerPlacementPolicy =
|
||||||
(excludedNodes, nodesRequired, sizeRequired) -> listOfDatanodeDetails
|
(excludedNodes, nodesRequired, sizeRequired) -> listOfDatanodeDetails
|
||||||
.subList(2, 2 + nodesRequired);
|
.subList(2, 2 + nodesRequired);
|
||||||
|
|
||||||
containerStateManager = Mockito.mock(ContainerStateManager.class);
|
containerManager = Mockito.mock(ContainerManager.class);
|
||||||
|
|
||||||
ContainerInfo containerInfo = new ContainerInfo.Builder()
|
ContainerInfo containerInfo = new ContainerInfo.Builder()
|
||||||
.setState(LifeCycleState.CLOSED)
|
.setState(LifeCycleState.CLOSED)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
when(containerStateManager.getContainer(anyObject()))
|
when(containerManager.getContainer(anyObject()))
|
||||||
.thenReturn(containerInfo);
|
.thenReturn(containerInfo);
|
||||||
|
|
||||||
when(containerStateManager.getContainerReplicas(new ContainerID(1L)))
|
when(containerManager.getContainerReplicas(new ContainerID(1L)))
|
||||||
.thenReturn(new HashSet<>(Arrays.asList(
|
.thenReturn(new HashSet<>(Arrays.asList(
|
||||||
listOfDatanodeDetails.get(0),
|
listOfContainerReplica.get(0),
|
||||||
listOfDatanodeDetails.get(1)
|
listOfContainerReplica.get(1)
|
||||||
)));
|
)));
|
||||||
|
|
||||||
|
|
||||||
when(containerStateManager.getContainerReplicas(new ContainerID(3L)))
|
when(containerManager.getContainerReplicas(new ContainerID(3L)))
|
||||||
.thenReturn(new HashSet<>());
|
.thenReturn(new HashSet<>());
|
||||||
|
|
||||||
queue = new EventQueue();
|
queue = new EventQueue();
|
||||||
|
@ -115,7 +121,7 @@ public class TestReplicationManager {
|
||||||
leaseManager = new LeaseManager<>("Test", 100000L);
|
leaseManager = new LeaseManager<>("Test", 100000L);
|
||||||
|
|
||||||
replicationManager = new ReplicationManager(containerPlacementPolicy,
|
replicationManager = new ReplicationManager(containerPlacementPolicy,
|
||||||
containerStateManager, queue, leaseManager);
|
containerManager, queue, leaseManager);
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
@ -182,7 +188,7 @@ public class TestReplicationManager {
|
||||||
new LeaseManager<>("Test", 1000L);
|
new LeaseManager<>("Test", 1000L);
|
||||||
|
|
||||||
replicationManager = new ReplicationManager(containerPlacementPolicy,
|
replicationManager = new ReplicationManager(containerPlacementPolicy,
|
||||||
containerStateManager, queue, rapidLeaseManager);
|
containerManager, queue, rapidLeaseManager);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
rapidLeaseManager.start();
|
rapidLeaseManager.start();
|
||||||
|
|
|
@ -102,7 +102,7 @@ public class TestContainerPlacement {
|
||||||
EventQueue eventQueue = new EventQueue();
|
EventQueue eventQueue = new EventQueue();
|
||||||
final int cacheSize = config.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
|
final int cacheSize = config.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
|
||||||
OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
|
OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
|
||||||
return new SCMContainerManager(config, scmNodeManager, cacheSize,
|
return new SCMContainerManager(config, scmNodeManager,
|
||||||
eventQueue);
|
eventQueue);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,38 +18,40 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hdds.scm.node;
|
package org.apache.hadoop.hdds.scm.node;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.UUID;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto;
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto;
|
||||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
|
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
|
||||||
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
|
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
|
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode;
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode;
|
||||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||||
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.mockito.ArgumentCaptor;
|
|
||||||
import static org.mockito.Matchers.eq;
|
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -57,86 +59,122 @@ import org.mockito.Mockito;
|
||||||
*/
|
*/
|
||||||
public class TestDeadNodeHandler {
|
public class TestDeadNodeHandler {
|
||||||
|
|
||||||
private List<ReplicationRequest> sentEvents = new ArrayList<>();
|
|
||||||
private SCMNodeManager nodeManager;
|
private SCMNodeManager nodeManager;
|
||||||
private ContainerStateManager containerStateManager;
|
private ContainerManager containerManager;
|
||||||
private NodeReportHandler nodeReportHandler;
|
private NodeReportHandler nodeReportHandler;
|
||||||
private DeadNodeHandler deadNodeHandler;
|
private DeadNodeHandler deadNodeHandler;
|
||||||
private EventPublisher publisher;
|
private EventPublisher publisher;
|
||||||
private EventQueue eventQueue;
|
private EventQueue eventQueue;
|
||||||
|
private String storageDir;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws IOException {
|
public void setup() throws IOException {
|
||||||
OzoneConfiguration conf = new OzoneConfiguration();
|
OzoneConfiguration conf = new OzoneConfiguration();
|
||||||
containerStateManager = new ContainerStateManager(conf,
|
storageDir = GenericTestUtils.getTempPath(
|
||||||
Mockito.mock(ContainerManager.class),
|
TestDeadNodeHandler.class.getSimpleName() + UUID.randomUUID());
|
||||||
Mockito.mock(PipelineSelector.class));
|
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, storageDir);
|
||||||
eventQueue = new EventQueue();
|
eventQueue = new EventQueue();
|
||||||
nodeManager = new SCMNodeManager(conf, "cluster1", null, eventQueue);
|
nodeManager = new SCMNodeManager(conf, "cluster1", null, eventQueue);
|
||||||
deadNodeHandler = new DeadNodeHandler(nodeManager,
|
containerManager = new SCMContainerManager(conf, nodeManager, eventQueue);
|
||||||
containerStateManager);
|
deadNodeHandler = new DeadNodeHandler(nodeManager, containerManager);
|
||||||
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
|
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
|
||||||
publisher = Mockito.mock(EventPublisher.class);
|
publisher = Mockito.mock(EventPublisher.class);
|
||||||
nodeReportHandler = new NodeReportHandler(nodeManager);
|
nodeReportHandler = new NodeReportHandler(nodeManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void teardown() {
|
||||||
|
FileUtil.fullyDelete(new File(storageDir));
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testOnMessage() throws IOException {
|
public void testOnMessage() throws IOException {
|
||||||
//GIVEN
|
//GIVEN
|
||||||
DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails();
|
||||||
DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails();
|
||||||
|
DatanodeDetails datanode3 = TestUtils.randomDatanodeDetails();
|
||||||
|
|
||||||
|
String storagePath = GenericTestUtils.getRandomizedTempPath()
|
||||||
|
.concat("/" + datanode1.getUuidString());
|
||||||
|
|
||||||
|
StorageReportProto storageOne = TestUtils.createStorageReport(
|
||||||
|
datanode1.getUuid(), storagePath, 100, 10, 90, null);
|
||||||
|
|
||||||
|
// Standalone pipeline now excludes the nodes which are already used,
|
||||||
|
// is the a proper behavior. Adding 9 datanodes for now to make the
|
||||||
|
// test case happy.
|
||||||
|
|
||||||
|
nodeManager.register(datanode1,
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(datanode2,
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(datanode3,
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
|
||||||
|
nodeManager.register(TestUtils.randomDatanodeDetails(),
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(TestUtils.randomDatanodeDetails(),
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(TestUtils.randomDatanodeDetails(),
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
|
||||||
|
nodeManager.register(TestUtils.randomDatanodeDetails(),
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(TestUtils.randomDatanodeDetails(),
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(TestUtils.randomDatanodeDetails(),
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
|
||||||
ContainerInfo container1 =
|
ContainerInfo container1 =
|
||||||
TestUtils.allocateContainer(containerStateManager);
|
TestUtils.allocateContainer(containerManager);
|
||||||
ContainerInfo container2 =
|
ContainerInfo container2 =
|
||||||
TestUtils.allocateContainer(containerStateManager);
|
TestUtils.allocateContainer(containerManager);
|
||||||
ContainerInfo container3 =
|
ContainerInfo container3 =
|
||||||
TestUtils.allocateContainer(containerStateManager);
|
TestUtils.allocateContainer(containerManager);
|
||||||
|
|
||||||
|
containerManager.updateContainerState(
|
||||||
|
container1.containerID(), HddsProtos.LifeCycleEvent.CREATE);
|
||||||
|
containerManager.updateContainerState(
|
||||||
|
container1.containerID(), HddsProtos.LifeCycleEvent.CREATED);
|
||||||
|
|
||||||
|
containerManager.updateContainerState(
|
||||||
|
container2.containerID(), HddsProtos.LifeCycleEvent.CREATE);
|
||||||
|
containerManager.updateContainerState(
|
||||||
|
container2.containerID(), HddsProtos.LifeCycleEvent.CREATED);
|
||||||
|
|
||||||
|
containerManager.updateContainerState(
|
||||||
|
container3.containerID(), HddsProtos.LifeCycleEvent.CREATE);
|
||||||
|
containerManager.updateContainerState(
|
||||||
|
container3.containerID(), HddsProtos.LifeCycleEvent.CREATED);
|
||||||
|
|
||||||
registerReplicas(datanode1, container1, container2);
|
registerReplicas(datanode1, container1, container2);
|
||||||
registerReplicas(datanode2, container1, container3);
|
registerReplicas(datanode2, container1, container3);
|
||||||
|
|
||||||
registerReplicas(containerStateManager, container1, datanode1, datanode2);
|
registerReplicas(containerManager, container1, datanode1, datanode2);
|
||||||
registerReplicas(containerStateManager, container2, datanode1);
|
registerReplicas(containerManager, container2, datanode1);
|
||||||
registerReplicas(containerStateManager, container3, datanode2);
|
registerReplicas(containerManager, container3, datanode2);
|
||||||
|
|
||||||
TestUtils.closeContainer(containerStateManager, container1);
|
TestUtils.closeContainer(containerManager, container1.containerID());
|
||||||
|
TestUtils.closeContainer(containerManager, container2.containerID());
|
||||||
|
TestUtils.closeContainer(containerManager, container3.containerID());
|
||||||
|
|
||||||
deadNodeHandler.onMessage(datanode1, publisher);
|
deadNodeHandler.onMessage(datanode1, publisher);
|
||||||
|
|
||||||
Set<DatanodeDetails> container1Replicas =
|
Set<ContainerReplica> container1Replicas = containerManager
|
||||||
containerStateManager.getContainerStateMap()
|
.getContainerReplicas(new ContainerID(container1.getContainerID()));
|
||||||
.getContainerReplicas(new ContainerID(container1.getContainerID()));
|
|
||||||
Assert.assertEquals(1, container1Replicas.size());
|
Assert.assertEquals(1, container1Replicas.size());
|
||||||
Assert.assertEquals(datanode2, container1Replicas.iterator().next());
|
Assert.assertEquals(datanode2,
|
||||||
|
container1Replicas.iterator().next().getDatanodeDetails());
|
||||||
|
|
||||||
Set<DatanodeDetails> container2Replicas =
|
Set<ContainerReplica> container2Replicas = containerManager
|
||||||
containerStateManager.getContainerStateMap()
|
.getContainerReplicas(new ContainerID(container2.getContainerID()));
|
||||||
.getContainerReplicas(new ContainerID(container2.getContainerID()));
|
|
||||||
Assert.assertEquals(0, container2Replicas.size());
|
Assert.assertEquals(0, container2Replicas.size());
|
||||||
|
|
||||||
Set<DatanodeDetails> container3Replicas =
|
Set<ContainerReplica> container3Replicas = containerManager
|
||||||
containerStateManager.getContainerStateMap()
|
|
||||||
.getContainerReplicas(new ContainerID(container3.getContainerID()));
|
.getContainerReplicas(new ContainerID(container3.getContainerID()));
|
||||||
Assert.assertEquals(1, container3Replicas.size());
|
Assert.assertEquals(1, container3Replicas.size());
|
||||||
Assert.assertEquals(datanode2, container3Replicas.iterator().next());
|
Assert.assertEquals(datanode2,
|
||||||
|
container3Replicas.iterator().next().getDatanodeDetails());
|
||||||
ArgumentCaptor<ReplicationRequest> replicationRequestParameter =
|
|
||||||
ArgumentCaptor.forClass(ReplicationRequest.class);
|
|
||||||
|
|
||||||
Mockito.verify(publisher)
|
|
||||||
.fireEvent(eq(SCMEvents.REPLICATE_CONTAINER),
|
|
||||||
replicationRequestParameter.capture());
|
|
||||||
|
|
||||||
Assert
|
|
||||||
.assertEquals(container1.getContainerID(),
|
|
||||||
replicationRequestParameter.getValue().getContainerId());
|
|
||||||
Assert
|
|
||||||
.assertEquals(1,
|
|
||||||
replicationRequestParameter.getValue().getReplicationCount());
|
|
||||||
Assert
|
|
||||||
.assertEquals(3,
|
|
||||||
replicationRequestParameter.getValue().getExpecReplicationCount());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -144,6 +182,7 @@ public class TestDeadNodeHandler {
|
||||||
//GIVEN
|
//GIVEN
|
||||||
DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails();
|
||||||
DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails();
|
||||||
|
|
||||||
String storagePath1 = GenericTestUtils.getRandomizedTempPath()
|
String storagePath1 = GenericTestUtils.getRandomizedTempPath()
|
||||||
.concat("/" + datanode1.getUuidString());
|
.concat("/" + datanode1.getUuidString());
|
||||||
String storagePath2 = GenericTestUtils.getRandomizedTempPath()
|
String storagePath2 = GenericTestUtils.getRandomizedTempPath()
|
||||||
|
@ -153,15 +192,17 @@ public class TestDeadNodeHandler {
|
||||||
datanode1.getUuid(), storagePath1, 100, 10, 90, null);
|
datanode1.getUuid(), storagePath1, 100, 10, 90, null);
|
||||||
StorageReportProto storageTwo = TestUtils.createStorageReport(
|
StorageReportProto storageTwo = TestUtils.createStorageReport(
|
||||||
datanode2.getUuid(), storagePath2, 200, 20, 180, null);
|
datanode2.getUuid(), storagePath2, 200, 20, 180, null);
|
||||||
|
|
||||||
|
nodeManager.register(datanode1,
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(datanode2,
|
||||||
|
TestUtils.createNodeReport(storageTwo), null);
|
||||||
|
|
||||||
nodeReportHandler.onMessage(getNodeReport(datanode1, storageOne),
|
nodeReportHandler.onMessage(getNodeReport(datanode1, storageOne),
|
||||||
Mockito.mock(EventPublisher.class));
|
Mockito.mock(EventPublisher.class));
|
||||||
nodeReportHandler.onMessage(getNodeReport(datanode2, storageTwo),
|
nodeReportHandler.onMessage(getNodeReport(datanode2, storageTwo),
|
||||||
Mockito.mock(EventPublisher.class));
|
Mockito.mock(EventPublisher.class));
|
||||||
|
|
||||||
ContainerInfo container1 =
|
|
||||||
TestUtils.allocateContainer(containerStateManager);
|
|
||||||
registerReplicas(datanode1, container1);
|
|
||||||
|
|
||||||
SCMNodeStat stat = nodeManager.getStats();
|
SCMNodeStat stat = nodeManager.getStats();
|
||||||
Assert.assertTrue(stat.getCapacity().get() == 300);
|
Assert.assertTrue(stat.getCapacity().get() == 300);
|
||||||
Assert.assertTrue(stat.getRemaining().get() == 270);
|
Assert.assertTrue(stat.getRemaining().get() == 270);
|
||||||
|
@ -190,32 +231,56 @@ public class TestDeadNodeHandler {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testOnMessageReplicaFailure() throws Exception {
|
public void testOnMessageReplicaFailure() throws Exception {
|
||||||
|
|
||||||
|
DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails();
|
||||||
|
DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails();
|
||||||
|
DatanodeDetails datanode3 = TestUtils.randomDatanodeDetails();
|
||||||
|
|
||||||
|
String storagePath = GenericTestUtils.getRandomizedTempPath()
|
||||||
|
.concat("/" + datanode1.getUuidString());
|
||||||
|
|
||||||
|
StorageReportProto storageOne = TestUtils.createStorageReport(
|
||||||
|
datanode1.getUuid(), storagePath, 100, 10, 90, null);
|
||||||
|
|
||||||
|
nodeManager.register(datanode1,
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(datanode2,
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
nodeManager.register(datanode3,
|
||||||
|
TestUtils.createNodeReport(storageOne), null);
|
||||||
|
|
||||||
DatanodeDetails dn1 = TestUtils.randomDatanodeDetails();
|
DatanodeDetails dn1 = TestUtils.randomDatanodeDetails();
|
||||||
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
||||||
.captureLogs(DeadNodeHandler.getLogger());
|
.captureLogs(DeadNodeHandler.getLogger());
|
||||||
String storagePath1 = GenericTestUtils.getRandomizedTempPath()
|
|
||||||
.concat("/" + dn1.getUuidString());
|
|
||||||
|
|
||||||
StorageReportProto storageOne = TestUtils.createStorageReport(
|
|
||||||
dn1.getUuid(), storagePath1, 100, 10, 90, null);
|
|
||||||
nodeReportHandler.onMessage(getNodeReport(dn1, storageOne),
|
nodeReportHandler.onMessage(getNodeReport(dn1, storageOne),
|
||||||
Mockito.mock(EventPublisher.class));
|
Mockito.mock(EventPublisher.class));
|
||||||
|
|
||||||
ContainerInfo container1 =
|
ContainerInfo container1 =
|
||||||
TestUtils.allocateContainer(containerStateManager);
|
TestUtils.allocateContainer(containerManager);
|
||||||
|
containerManager.updateContainerState(
|
||||||
|
container1.containerID(), HddsProtos.LifeCycleEvent.CREATE);
|
||||||
|
containerManager.updateContainerState(
|
||||||
|
container1.containerID(), HddsProtos.LifeCycleEvent.CREATED);
|
||||||
|
TestUtils.closeContainer(containerManager, container1.containerID());
|
||||||
|
|
||||||
registerReplicas(dn1, container1);
|
registerReplicas(dn1, container1);
|
||||||
|
|
||||||
deadNodeHandler.onMessage(dn1, eventQueue);
|
deadNodeHandler.onMessage(dn1, eventQueue);
|
||||||
Assert.assertTrue(logCapturer.getOutput().contains(
|
Assert.assertTrue(logCapturer.getOutput().contains(
|
||||||
"DataNode " + dn1.getUuid() + " doesn't have replica for container "
|
"Exception while removing container replica "));
|
||||||
+ container1.getContainerID()));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void registerReplicas(ContainerStateManager csm,
|
private void registerReplicas(ContainerManager containerManager,
|
||||||
ContainerInfo container, DatanodeDetails... datanodes) {
|
ContainerInfo container, DatanodeDetails... datanodes)
|
||||||
csm.getContainerStateMap()
|
throws ContainerNotFoundException {
|
||||||
.addContainerReplica(new ContainerID(container.getContainerID()),
|
for (DatanodeDetails datanode : datanodes) {
|
||||||
datanodes);
|
containerManager.updateContainerReplica(
|
||||||
|
new ContainerID(container.getContainerID()),
|
||||||
|
ContainerReplica.newBuilder()
|
||||||
|
.setContainerID(container.containerID())
|
||||||
|
.setDatanodeDetails(datanode).build());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void registerReplicas(DatanodeDetails datanode,
|
private void registerReplicas(DatanodeDetails datanode,
|
||||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.HddsTestUtils;
|
import org.apache.hadoop.hdds.scm.HddsTestUtils;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler
|
||||||
import org.apache.hadoop.hdds.scm.container.CloseContainerWatcher;
|
import org.apache.hadoop.hdds.scm.container.CloseContainerWatcher;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
@ -274,10 +274,12 @@ public class TestCloseContainerWatcher implements EventHandler<ContainerID> {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
ContainerInfo containerInfo = Mockito.mock(ContainerInfo.class);
|
ContainerInfo containerInfo = Mockito.mock(ContainerInfo.class);
|
||||||
ContainerInfo containerInfo2 = Mockito.mock(ContainerInfo.class);
|
ContainerInfo containerInfo2 = Mockito.mock(ContainerInfo.class);
|
||||||
when(containerManager.getContainer(id1)).thenReturn(containerInfo);
|
when(containerManager.getContainer(ContainerID.valueof(id1)))
|
||||||
when(containerManager.getContainer(id2)).thenReturn(containerInfo2);
|
.thenReturn(containerInfo);
|
||||||
when(containerInfo.isContainerOpen()).thenReturn(true);
|
when(containerManager.getContainer(ContainerID.valueof(id2)))
|
||||||
when(containerInfo2.isContainerOpen()).thenReturn(isOpen);
|
.thenReturn(containerInfo2);
|
||||||
|
when(containerInfo.isOpen()).thenReturn(true);
|
||||||
|
when(containerInfo2.isOpen()).thenReturn(isOpen);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.hadoop.hdds.scm.cli.container.InfoSubcommand;
|
||||||
import org.apache.hadoop.hdds.scm.cli.container.ListSubcommand;
|
import org.apache.hadoop.hdds.scm.cli.container.ListSubcommand;
|
||||||
import org.apache.hadoop.hdds.scm.client.ContainerOperationClient;
|
import org.apache.hadoop.hdds.scm.client.ContainerOperationClient;
|
||||||
import org.apache.hadoop.hdds.scm.client.ScmClient;
|
import org.apache.hadoop.hdds.scm.client.ScmClient;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB
|
import org.apache.hadoop.hdds.scm.protocolPB
|
||||||
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
|
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
|
||||||
|
|
|
@ -24,7 +24,7 @@ import java.util.concurrent.Callable;
|
||||||
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
|
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
|
||||||
import org.apache.hadoop.hdds.scm.cli.SCMCLI;
|
import org.apache.hadoop.hdds.scm.cli.SCMCLI;
|
||||||
import org.apache.hadoop.hdds.scm.client.ScmClient;
|
import org.apache.hadoop.hdds.scm.client.ScmClient;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.ozone.web.utils.JsonUtils;
|
import org.apache.hadoop.ozone.web.utils.JsonUtils;
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -52,7 +52,7 @@ public class ListSubcommand implements Callable<Void> {
|
||||||
|
|
||||||
@Option(names = {"-s", "--start"},
|
@Option(names = {"-s", "--start"},
|
||||||
description = "Container id to start the iteration", required = true)
|
description = "Container id to start the iteration", required = true)
|
||||||
private long startId;
|
private long startId = 1;
|
||||||
|
|
||||||
@Option(names = {"-c", "--count"},
|
@Option(names = {"-c", "--count"},
|
||||||
description = "Maximum number of containers to list",
|
description = "Maximum number of containers to list",
|
||||||
|
|
|
@ -21,7 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.fs.FSExceptionMessages;
|
import org.apache.hadoop.fs.FSExceptionMessages;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.io.retry.RetryPolicy;
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
||||||
|
|
|
@ -16,7 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hdds.scm.container;
|
package org.apache.hadoop.hdds.scm.container;
|
||||||
|
|
||||||
import com.google.common.primitives.Longs;
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import org.apache.commons.lang3.RandomUtils;
|
import org.apache.commons.lang3.RandomUtils;
|
||||||
|
@ -24,26 +23,22 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
||||||
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.test.LambdaTestUtils;
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.NavigableSet;
|
import java.util.NavigableSet;
|
||||||
import java.util.Random;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
import org.slf4j.event.Level;
|
import org.slf4j.event.Level;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -57,7 +52,6 @@ public class TestContainerStateManagerIntegration {
|
||||||
private StorageContainerManager scm;
|
private StorageContainerManager scm;
|
||||||
private ContainerManager containerManager;
|
private ContainerManager containerManager;
|
||||||
private ContainerStateManager containerStateManager;
|
private ContainerStateManager containerStateManager;
|
||||||
private PipelineSelector selector;
|
|
||||||
private String containerOwner = "OZONE";
|
private String containerOwner = "OZONE";
|
||||||
|
|
||||||
|
|
||||||
|
@ -70,8 +64,8 @@ public class TestContainerStateManagerIntegration {
|
||||||
xceiverClientManager = new XceiverClientManager(conf);
|
xceiverClientManager = new XceiverClientManager(conf);
|
||||||
scm = cluster.getStorageContainerManager();
|
scm = cluster.getStorageContainerManager();
|
||||||
containerManager = scm.getContainerManager();
|
containerManager = scm.getContainerManager();
|
||||||
containerStateManager = containerManager.getStateManager();
|
containerStateManager = ((SCMContainerManager)containerManager)
|
||||||
selector = containerManager.getPipelineSelector();
|
.getContainerStateManager();
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -88,13 +82,13 @@ public class TestContainerStateManagerIntegration {
|
||||||
.allocateContainer(
|
.allocateContainer(
|
||||||
xceiverClientManager.getType(),
|
xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), containerOwner);
|
xceiverClientManager.getFactor(), containerOwner);
|
||||||
|
ContainerStateManager stateManager = new ContainerStateManager(conf);
|
||||||
ContainerInfo info = containerStateManager
|
ContainerInfo info = containerStateManager
|
||||||
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
|
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
HddsProtos.LifeCycleState.ALLOCATED);
|
HddsProtos.LifeCycleState.ALLOCATED);
|
||||||
Assert.assertEquals(container1.getContainerInfo().getContainerID(),
|
Assert.assertEquals(container1.getContainerInfo().getContainerID(),
|
||||||
info.getContainerID());
|
info.getContainerID());
|
||||||
Assert.assertEquals(OzoneConsts.GB * 3, info.getAllocatedBytes());
|
|
||||||
Assert.assertEquals(containerOwner, info.getOwner());
|
Assert.assertEquals(containerOwner, info.getOwner());
|
||||||
Assert.assertEquals(xceiverClientManager.getType(),
|
Assert.assertEquals(xceiverClientManager.getType(),
|
||||||
info.getReplicationType());
|
info.getReplicationType());
|
||||||
|
@ -117,35 +111,49 @@ public class TestContainerStateManagerIntegration {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testContainerStateManagerRestart() throws IOException {
|
public void testContainerStateManagerRestart()
|
||||||
|
throws IOException, TimeoutException, InterruptedException {
|
||||||
// Allocate 5 containers in ALLOCATED state and 5 in CREATING state
|
// Allocate 5 containers in ALLOCATED state and 5 in CREATING state
|
||||||
|
|
||||||
List<ContainerInfo> containers = new ArrayList<>();
|
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
|
|
||||||
ContainerWithPipeline container = scm.getClientProtocolServer()
|
ContainerWithPipeline container = scm.getClientProtocolServer()
|
||||||
.allocateContainer(
|
.allocateContainer(
|
||||||
xceiverClientManager.getType(),
|
xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), containerOwner);
|
xceiverClientManager.getFactor(), containerOwner);
|
||||||
containers.add(container.getContainerInfo());
|
|
||||||
if (i >= 5) {
|
if (i >= 5) {
|
||||||
scm.getContainerManager().updateContainerState(container
|
scm.getContainerManager().updateContainerState(container
|
||||||
.getContainerInfo().getContainerID(),
|
.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// New instance of ContainerStateManager should load all the containers in
|
cluster.restartStorageContainerManager();
|
||||||
// container store.
|
|
||||||
ContainerStateManager stateManager =
|
List<ContainerInfo> result = cluster.getStorageContainerManager()
|
||||||
new ContainerStateManager(conf, containerManager, selector);
|
.getContainerManager().listContainer(null, 100);
|
||||||
int matchCount = stateManager
|
|
||||||
.getMatchingContainerIDs(containerOwner,
|
long matchCount = result.stream()
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
.filter(info ->
|
||||||
HddsProtos.LifeCycleState.ALLOCATED).size();
|
info.getOwner().equals(containerOwner))
|
||||||
|
.filter(info ->
|
||||||
|
info.getReplicationType() == xceiverClientManager.getType())
|
||||||
|
.filter(info ->
|
||||||
|
info.getReplicationFactor() == xceiverClientManager.getFactor())
|
||||||
|
.filter(info ->
|
||||||
|
info.getState() == HddsProtos.LifeCycleState.ALLOCATED)
|
||||||
|
.count();
|
||||||
Assert.assertEquals(5, matchCount);
|
Assert.assertEquals(5, matchCount);
|
||||||
matchCount = stateManager.getMatchingContainerIDs(containerOwner,
|
matchCount = result.stream()
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
.filter(info ->
|
||||||
HddsProtos.LifeCycleState.CREATING).size();
|
info.getOwner().equals(containerOwner))
|
||||||
|
.filter(info ->
|
||||||
|
info.getReplicationType() == xceiverClientManager.getType())
|
||||||
|
.filter(info ->
|
||||||
|
info.getReplicationFactor() == xceiverClientManager.getFactor())
|
||||||
|
.filter(info ->
|
||||||
|
info.getState() == HddsProtos.LifeCycleState.CREATING)
|
||||||
|
.count();
|
||||||
Assert.assertEquals(5, matchCount);
|
Assert.assertEquals(5, matchCount);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -155,10 +163,10 @@ public class TestContainerStateManagerIntegration {
|
||||||
allocateContainer(xceiverClientManager.getType(),
|
allocateContainer(xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), containerOwner);
|
xceiverClientManager.getFactor(), containerOwner);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container1.getContainerInfo().getContainerID(),
|
.updateContainerState(container1.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container1.getContainerInfo().getContainerID(),
|
.updateContainerState(container1.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATED);
|
HddsProtos.LifeCycleEvent.CREATED);
|
||||||
|
|
||||||
ContainerWithPipeline container2 = scm.getClientProtocolServer().
|
ContainerWithPipeline container2 = scm.getClientProtocolServer().
|
||||||
|
@ -176,23 +184,24 @@ public class TestContainerStateManagerIntegration {
|
||||||
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
|
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
HddsProtos.LifeCycleState.ALLOCATED);
|
HddsProtos.LifeCycleState.ALLOCATED);
|
||||||
|
// space has already been allocated in container1, now container 2 should
|
||||||
|
// be chosen.
|
||||||
Assert.assertEquals(container2.getContainerInfo().getContainerID(),
|
Assert.assertEquals(container2.getContainerInfo().getContainerID(),
|
||||||
info.getContainerID());
|
info.getContainerID());
|
||||||
|
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container2.getContainerInfo().getContainerID(),
|
.updateContainerState(container2.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container2.getContainerInfo().getContainerID(),
|
.updateContainerState(container2.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATED);
|
HddsProtos.LifeCycleEvent.CREATED);
|
||||||
|
|
||||||
// space has already been allocated in container1, now container 2 should
|
// now we have to get container1
|
||||||
// be chosen.
|
|
||||||
info = containerStateManager
|
info = containerStateManager
|
||||||
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
|
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
HddsProtos.LifeCycleState.OPEN);
|
HddsProtos.LifeCycleState.OPEN);
|
||||||
Assert.assertEquals(container2.getContainerInfo().getContainerID(),
|
Assert.assertEquals(container1.getContainerInfo().getContainerID(),
|
||||||
info.getContainerID());
|
info.getContainerID());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -217,7 +226,7 @@ public class TestContainerStateManagerIntegration {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container1.getContainerInfo().getContainerID(),
|
.updateContainerState(container1.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -225,7 +234,7 @@ public class TestContainerStateManagerIntegration {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container1.getContainerInfo().getContainerID(),
|
.updateContainerState(container1.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATED);
|
HddsProtos.LifeCycleEvent.CREATED);
|
||||||
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -233,7 +242,7 @@ public class TestContainerStateManagerIntegration {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container1.getContainerInfo().getContainerID(),
|
.updateContainerState(container1.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.FINALIZE);
|
HddsProtos.LifeCycleEvent.FINALIZE);
|
||||||
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -241,7 +250,7 @@ public class TestContainerStateManagerIntegration {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container1.getContainerInfo().getContainerID(),
|
.updateContainerState(container1.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CLOSE);
|
HddsProtos.LifeCycleEvent.CLOSE);
|
||||||
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -249,7 +258,7 @@ public class TestContainerStateManagerIntegration {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container1.getContainerInfo().getContainerID(),
|
.updateContainerState(container1.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.DELETE);
|
HddsProtos.LifeCycleEvent.DELETE);
|
||||||
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -257,7 +266,7 @@ public class TestContainerStateManagerIntegration {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
|
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container1.getContainerInfo().getContainerID(),
|
.updateContainerState(container1.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CLEANUP);
|
HddsProtos.LifeCycleEvent.CLEANUP);
|
||||||
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -271,10 +280,10 @@ public class TestContainerStateManagerIntegration {
|
||||||
xceiverClientManager.getType(),
|
xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), containerOwner);
|
xceiverClientManager.getFactor(), containerOwner);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container2.getContainerInfo().getContainerID(),
|
.updateContainerState(container2.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container2.getContainerInfo().getContainerID(),
|
.updateContainerState(container2.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.TIMEOUT);
|
HddsProtos.LifeCycleEvent.TIMEOUT);
|
||||||
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -288,16 +297,16 @@ public class TestContainerStateManagerIntegration {
|
||||||
xceiverClientManager.getType(),
|
xceiverClientManager.getType(),
|
||||||
xceiverClientManager.getFactor(), containerOwner);
|
xceiverClientManager.getFactor(), containerOwner);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container3.getContainerInfo().getContainerID(),
|
.updateContainerState(container3.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
HddsProtos.LifeCycleEvent.CREATE);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container3.getContainerInfo().getContainerID(),
|
.updateContainerState(container3.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CREATED);
|
HddsProtos.LifeCycleEvent.CREATED);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container3.getContainerInfo().getContainerID(),
|
.updateContainerState(container3.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.FINALIZE);
|
HddsProtos.LifeCycleEvent.FINALIZE);
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(container3.getContainerInfo().getContainerID(),
|
.updateContainerState(container3.getContainerInfo().containerID(),
|
||||||
HddsProtos.LifeCycleEvent.CLOSE);
|
HddsProtos.LifeCycleEvent.CLOSE);
|
||||||
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
||||||
|
@ -305,46 +314,6 @@ public class TestContainerStateManagerIntegration {
|
||||||
Assert.assertEquals(1, containers);
|
Assert.assertEquals(1, containers);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testUpdatingAllocatedBytes() throws Exception {
|
|
||||||
ContainerWithPipeline container1 = scm.getClientProtocolServer()
|
|
||||||
.allocateContainer(xceiverClientManager.getType(),
|
|
||||||
xceiverClientManager.getFactor(), containerOwner);
|
|
||||||
containerManager.updateContainerState(container1
|
|
||||||
.getContainerInfo().getContainerID(),
|
|
||||||
HddsProtos.LifeCycleEvent.CREATE);
|
|
||||||
containerManager.updateContainerState(container1
|
|
||||||
.getContainerInfo().getContainerID(),
|
|
||||||
HddsProtos.LifeCycleEvent.CREATED);
|
|
||||||
|
|
||||||
Random ran = new Random();
|
|
||||||
long allocatedSize = 0;
|
|
||||||
for (int i = 0; i<5; i++) {
|
|
||||||
long size = Math.abs(ran.nextLong() % OzoneConsts.GB);
|
|
||||||
allocatedSize += size;
|
|
||||||
// trigger allocating bytes by calling getMatchingContainer
|
|
||||||
ContainerInfo info = containerStateManager
|
|
||||||
.getMatchingContainer(size, containerOwner,
|
|
||||||
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
|
|
||||||
HddsProtos.LifeCycleState.OPEN);
|
|
||||||
Assert.assertEquals(container1.getContainerInfo().getContainerID(),
|
|
||||||
info.getContainerID());
|
|
||||||
|
|
||||||
SCMContainerManager containerMapping =
|
|
||||||
(SCMContainerManager) containerManager;
|
|
||||||
// manually trigger a flush, this will persist the allocated bytes value
|
|
||||||
// to disk
|
|
||||||
containerMapping.flushContainerInfo();
|
|
||||||
|
|
||||||
// the persisted value should always be equal to allocated size.
|
|
||||||
byte[] containerBytes = containerMapping.getContainerStore().get(
|
|
||||||
Longs.toByteArray(container1.getContainerInfo().getContainerID()));
|
|
||||||
HddsProtos.SCMContainerInfo infoProto =
|
|
||||||
HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
|
|
||||||
ContainerInfo currentInfo = ContainerInfo.fromProtobuf(infoProto);
|
|
||||||
Assert.assertEquals(allocatedSize, currentInfo.getAllocatedBytes());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testReplicaMap() throws Exception {
|
public void testReplicaMap() throws Exception {
|
||||||
|
@ -360,59 +329,71 @@ public class TestContainerStateManagerIntegration {
|
||||||
|
|
||||||
// Test 1: no replica's exist
|
// Test 1: no replica's exist
|
||||||
ContainerID containerID = ContainerID.valueof(RandomUtils.nextLong());
|
ContainerID containerID = ContainerID.valueof(RandomUtils.nextLong());
|
||||||
Set<DatanodeDetails> replicaSet;
|
Set<ContainerReplica> replicaSet;
|
||||||
LambdaTestUtils.intercept(SCMException.class, "", () -> {
|
try {
|
||||||
containerStateManager.getContainerReplicas(containerID);
|
containerStateManager.getContainerReplicas(containerID);
|
||||||
});
|
Assert.fail();
|
||||||
|
} catch (ContainerNotFoundException ex) {
|
||||||
|
// expected.
|
||||||
|
}
|
||||||
|
|
||||||
|
ContainerWithPipeline container = scm.getClientProtocolServer()
|
||||||
|
.allocateContainer(
|
||||||
|
xceiverClientManager.getType(),
|
||||||
|
xceiverClientManager.getFactor(), containerOwner);
|
||||||
|
|
||||||
|
ContainerID id = container.getContainerInfo().containerID();
|
||||||
|
|
||||||
// Test 2: Add replica nodes and then test
|
// Test 2: Add replica nodes and then test
|
||||||
containerStateManager.addContainerReplica(containerID, dn1);
|
ContainerReplica replicaOne = ContainerReplica.newBuilder()
|
||||||
containerStateManager.addContainerReplica(containerID, dn2);
|
.setContainerID(id)
|
||||||
replicaSet = containerStateManager.getContainerReplicas(containerID);
|
.setDatanodeDetails(dn1)
|
||||||
|
.build();
|
||||||
|
ContainerReplica replicaTwo = ContainerReplica.newBuilder()
|
||||||
|
.setContainerID(id)
|
||||||
|
.setDatanodeDetails(dn2)
|
||||||
|
.build();
|
||||||
|
containerStateManager.updateContainerReplica(id, replicaOne);
|
||||||
|
containerStateManager.updateContainerReplica(id, replicaTwo);
|
||||||
|
replicaSet = containerStateManager.getContainerReplicas(id);
|
||||||
Assert.assertEquals(2, replicaSet.size());
|
Assert.assertEquals(2, replicaSet.size());
|
||||||
Assert.assertTrue(replicaSet.contains(dn1));
|
Assert.assertTrue(replicaSet.contains(replicaOne));
|
||||||
Assert.assertTrue(replicaSet.contains(dn2));
|
Assert.assertTrue(replicaSet.contains(replicaTwo));
|
||||||
|
|
||||||
// Test 3: Remove one replica node and then test
|
// Test 3: Remove one replica node and then test
|
||||||
containerStateManager.removeContainerReplica(containerID, dn1);
|
containerStateManager.removeContainerReplica(id, replicaOne);
|
||||||
replicaSet = containerStateManager.getContainerReplicas(containerID);
|
replicaSet = containerStateManager.getContainerReplicas(id);
|
||||||
Assert.assertEquals(1, replicaSet.size());
|
Assert.assertEquals(1, replicaSet.size());
|
||||||
Assert.assertFalse(replicaSet.contains(dn1));
|
Assert.assertFalse(replicaSet.contains(replicaOne));
|
||||||
Assert.assertTrue(replicaSet.contains(dn2));
|
Assert.assertTrue(replicaSet.contains(replicaTwo));
|
||||||
|
|
||||||
// Test 3: Remove second replica node and then test
|
// Test 3: Remove second replica node and then test
|
||||||
containerStateManager.removeContainerReplica(containerID, dn2);
|
containerStateManager.removeContainerReplica(id, replicaTwo);
|
||||||
replicaSet = containerStateManager.getContainerReplicas(containerID);
|
replicaSet = containerStateManager.getContainerReplicas(id);
|
||||||
Assert.assertEquals(0, replicaSet.size());
|
Assert.assertEquals(0, replicaSet.size());
|
||||||
Assert.assertFalse(replicaSet.contains(dn1));
|
Assert.assertFalse(replicaSet.contains(replicaOne));
|
||||||
Assert.assertFalse(replicaSet.contains(dn2));
|
Assert.assertFalse(replicaSet.contains(replicaTwo));
|
||||||
|
|
||||||
// Test 4: Re-insert dn1
|
// Test 4: Re-insert dn1
|
||||||
containerStateManager.addContainerReplica(containerID, dn1);
|
containerStateManager.updateContainerReplica(id, replicaOne);
|
||||||
replicaSet = containerStateManager.getContainerReplicas(containerID);
|
replicaSet = containerStateManager.getContainerReplicas(id);
|
||||||
Assert.assertEquals(1, replicaSet.size());
|
Assert.assertEquals(1, replicaSet.size());
|
||||||
Assert.assertTrue(replicaSet.contains(dn1));
|
Assert.assertTrue(replicaSet.contains(replicaOne));
|
||||||
Assert.assertFalse(replicaSet.contains(dn2));
|
Assert.assertFalse(replicaSet.contains(replicaTwo));
|
||||||
|
|
||||||
// Re-insert dn2
|
// Re-insert dn2
|
||||||
containerStateManager.addContainerReplica(containerID, dn2);
|
containerStateManager.updateContainerReplica(id, replicaTwo);
|
||||||
replicaSet = containerStateManager.getContainerReplicas(containerID);
|
replicaSet = containerStateManager.getContainerReplicas(id);
|
||||||
Assert.assertEquals(2, replicaSet.size());
|
Assert.assertEquals(2, replicaSet.size());
|
||||||
Assert.assertTrue(replicaSet.contains(dn1));
|
Assert.assertTrue(replicaSet.contains(replicaOne));
|
||||||
Assert.assertTrue(replicaSet.contains(dn2));
|
Assert.assertTrue(replicaSet.contains(replicaTwo));
|
||||||
|
|
||||||
Assert.assertFalse(logCapturer.getOutput().contains(
|
|
||||||
"ReplicaMap already contains entry for container Id: " + containerID
|
|
||||||
.toString() + ",DataNode: " + dn1.toString()));
|
|
||||||
// Re-insert dn1
|
// Re-insert dn1
|
||||||
containerStateManager.addContainerReplica(containerID, dn1);
|
containerStateManager.updateContainerReplica(id, replicaOne);
|
||||||
replicaSet = containerStateManager.getContainerReplicas(containerID);
|
replicaSet = containerStateManager.getContainerReplicas(id);
|
||||||
Assert.assertEquals(2, replicaSet.size());
|
Assert.assertEquals(2, replicaSet.size());
|
||||||
Assert.assertTrue(replicaSet.contains(dn1));
|
Assert.assertTrue(replicaSet.contains(replicaOne));
|
||||||
Assert.assertTrue(replicaSet.contains(dn2));
|
Assert.assertTrue(replicaSet.contains(replicaTwo));
|
||||||
Assert.assertTrue(logCapturer.getOutput().contains(
|
|
||||||
"ReplicaMap already contains entry for container Id: " + containerID
|
|
||||||
.toString() + ",DataNode: " + dn1.toString()));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
||||||
.ContainerWithPipeline;
|
.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
||||||
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
||||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||||
|
@ -50,7 +49,6 @@ public class TestNode2PipelineMap {
|
||||||
private static OzoneConfiguration conf;
|
private static OzoneConfiguration conf;
|
||||||
private static StorageContainerManager scm;
|
private static StorageContainerManager scm;
|
||||||
private static ContainerWithPipeline ratisContainer;
|
private static ContainerWithPipeline ratisContainer;
|
||||||
private static ContainerStateMap stateMap;
|
|
||||||
private static ContainerManager containerManager;
|
private static ContainerManager containerManager;
|
||||||
private static PipelineSelector pipelineSelector;
|
private static PipelineSelector pipelineSelector;
|
||||||
|
|
||||||
|
@ -66,7 +64,6 @@ public class TestNode2PipelineMap {
|
||||||
cluster.waitForClusterToBeReady();
|
cluster.waitForClusterToBeReady();
|
||||||
scm = cluster.getStorageContainerManager();
|
scm = cluster.getStorageContainerManager();
|
||||||
containerManager = scm.getContainerManager();
|
containerManager = scm.getContainerManager();
|
||||||
stateMap = containerManager.getStateManager().getContainerStateMap();
|
|
||||||
ratisContainer = containerManager.allocateContainer(
|
ratisContainer = containerManager.allocateContainer(
|
||||||
RATIS, THREE, "testOwner");
|
RATIS, THREE, "testOwner");
|
||||||
pipelineSelector = containerManager.getPipelineSelector();
|
pipelineSelector = containerManager.getPipelineSelector();
|
||||||
|
@ -89,10 +86,10 @@ public class TestNode2PipelineMap {
|
||||||
Set<ContainerID> set = pipelineSelector.getOpenContainerIDsByPipeline(
|
Set<ContainerID> set = pipelineSelector.getOpenContainerIDsByPipeline(
|
||||||
ratisContainer.getPipeline().getId());
|
ratisContainer.getPipeline().getId());
|
||||||
|
|
||||||
long cId = ratisContainer.getContainerInfo().getContainerID();
|
ContainerID cId = ratisContainer.getContainerInfo().containerID();
|
||||||
Assert.assertEquals(1, set.size());
|
Assert.assertEquals(1, set.size());
|
||||||
set.forEach(containerID ->
|
set.forEach(containerID ->
|
||||||
Assert.assertEquals(containerID, ContainerID.valueof(cId)));
|
Assert.assertEquals(containerID, cId));
|
||||||
|
|
||||||
List<DatanodeDetails> dns = ratisContainer.getPipeline().getMachines();
|
List<DatanodeDetails> dns = ratisContainer.getPipeline().getMachines();
|
||||||
Assert.assertEquals(3, dns.size());
|
Assert.assertEquals(3, dns.size());
|
||||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
||||||
.ContainerWithPipeline;
|
.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
|
||||||
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
|
||||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
|
@ -50,7 +49,6 @@ public class TestPipelineClose {
|
||||||
private static StorageContainerManager scm;
|
private static StorageContainerManager scm;
|
||||||
private static ContainerWithPipeline ratisContainer1;
|
private static ContainerWithPipeline ratisContainer1;
|
||||||
private static ContainerWithPipeline ratisContainer2;
|
private static ContainerWithPipeline ratisContainer2;
|
||||||
private static ContainerStateMap stateMap;
|
|
||||||
private static ContainerManager containerManager;
|
private static ContainerManager containerManager;
|
||||||
private static PipelineSelector pipelineSelector;
|
private static PipelineSelector pipelineSelector;
|
||||||
|
|
||||||
|
@ -66,7 +64,6 @@ public class TestPipelineClose {
|
||||||
cluster.waitForClusterToBeReady();
|
cluster.waitForClusterToBeReady();
|
||||||
scm = cluster.getStorageContainerManager();
|
scm = cluster.getStorageContainerManager();
|
||||||
containerManager = scm.getContainerManager();
|
containerManager = scm.getContainerManager();
|
||||||
stateMap = containerManager.getStateManager().getContainerStateMap();
|
|
||||||
ratisContainer1 = containerManager
|
ratisContainer1 = containerManager
|
||||||
.allocateContainer(RATIS, THREE, "testOwner");
|
.allocateContainer(RATIS, THREE, "testOwner");
|
||||||
ratisContainer2 = containerManager
|
ratisContainer2 = containerManager
|
||||||
|
@ -93,10 +90,9 @@ public class TestPipelineClose {
|
||||||
Set<ContainerID> set = pipelineSelector.getOpenContainerIDsByPipeline(
|
Set<ContainerID> set = pipelineSelector.getOpenContainerIDsByPipeline(
|
||||||
ratisContainer1.getPipeline().getId());
|
ratisContainer1.getPipeline().getId());
|
||||||
|
|
||||||
long cId = ratisContainer1.getContainerInfo().getContainerID();
|
ContainerID cId = ratisContainer1.getContainerInfo().containerID();
|
||||||
Assert.assertEquals(1, set.size());
|
Assert.assertEquals(1, set.size());
|
||||||
set.forEach(containerID ->
|
set.forEach(containerID -> Assert.assertEquals(containerID, cId));
|
||||||
Assert.assertEquals(containerID, ContainerID.valueof(cId)));
|
|
||||||
|
|
||||||
// Now close the container and it should not show up while fetching
|
// Now close the container and it should not show up while fetching
|
||||||
// containers by pipeline
|
// containers by pipeline
|
||||||
|
@ -133,7 +129,7 @@ public class TestPipelineClose {
|
||||||
ratisContainer2.getPipeline().getId());
|
ratisContainer2.getPipeline().getId());
|
||||||
Assert.assertEquals(1, setOpen.size());
|
Assert.assertEquals(1, setOpen.size());
|
||||||
|
|
||||||
long cId2 = ratisContainer2.getContainerInfo().getContainerID();
|
ContainerID cId2 = ratisContainer2.getContainerInfo().containerID();
|
||||||
containerManager
|
containerManager
|
||||||
.updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATE);
|
.updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATE);
|
||||||
containerManager
|
containerManager
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.ozone;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||||
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
||||||
|
@ -45,14 +46,15 @@ public class OzoneTestUtils {
|
||||||
return performOperationOnKeyContainers((blockID) -> {
|
return performOperationOnKeyContainers((blockID) -> {
|
||||||
try {
|
try {
|
||||||
scm.getContainerManager()
|
scm.getContainerManager()
|
||||||
.updateContainerState(blockID.getContainerID(),
|
.updateContainerState(ContainerID.valueof(blockID.getContainerID()),
|
||||||
HddsProtos.LifeCycleEvent.FINALIZE);
|
HddsProtos.LifeCycleEvent.FINALIZE);
|
||||||
scm.getContainerManager()
|
scm.getContainerManager()
|
||||||
.updateContainerState(blockID.getContainerID(),
|
.updateContainerState(ContainerID.valueof(blockID.getContainerID()),
|
||||||
HddsProtos.LifeCycleEvent.CLOSE);
|
HddsProtos.LifeCycleEvent.CLOSE);
|
||||||
Assert.assertFalse(scm.getContainerManager()
|
Assert.assertFalse(scm.getContainerManager()
|
||||||
.getContainerWithPipeline(blockID.getContainerID())
|
.getContainerWithPipeline(ContainerID.valueof(
|
||||||
.getContainerInfo().isContainerOpen());
|
blockID.getContainerID()))
|
||||||
|
.getContainerInfo().isOpen());
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
e.printStackTrace();
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
|
import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
|
||||||
import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
|
import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||||
|
@ -168,8 +169,7 @@ public class TestStorageContainerManager {
|
||||||
} else {
|
} else {
|
||||||
// If passes permission check, it should fail with
|
// If passes permission check, it should fail with
|
||||||
// key not exist exception.
|
// key not exist exception.
|
||||||
Assert.assertTrue(e.getMessage()
|
Assert.assertTrue(e instanceof ContainerNotFoundException);
|
||||||
.contains("Specified key does not exist"));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.StorageType;
|
import org.apache.hadoop.hdds.protocol.StorageType;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.ozone.*;
|
import org.apache.hadoop.ozone.*;
|
||||||
import org.apache.hadoop.ozone.client.*;
|
import org.apache.hadoop.ozone.client.*;
|
||||||
|
@ -445,7 +446,8 @@ public class TestOzoneRestClient {
|
||||||
// Sum the data size from chunks in Container via containerID
|
// Sum the data size from chunks in Container via containerID
|
||||||
// and localID, make sure the size equals to the actually value size.
|
// and localID, make sure the size equals to the actually value size.
|
||||||
Pipeline pipeline = cluster.getStorageContainerManager()
|
Pipeline pipeline = cluster.getStorageContainerManager()
|
||||||
.getContainerManager().getContainerWithPipeline(containerID)
|
.getContainerManager().getContainerWithPipeline(
|
||||||
|
ContainerID.valueof(containerID))
|
||||||
.getPipeline();
|
.getPipeline();
|
||||||
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
||||||
Assert.assertEquals(datanodes.size(), 1);
|
Assert.assertEquals(datanodes.size(), 1);
|
||||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.ozone.HddsDatanodeService;
|
import org.apache.hadoop.ozone.HddsDatanodeService;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.
|
import org.apache.hadoop.hdds.scm.container.common.helpers.
|
||||||
StorageContainerException;
|
StorageContainerException;
|
||||||
|
@ -302,7 +303,8 @@ public class TestCloseContainerHandlingByClient {
|
||||||
for (long containerID : containerIdList) {
|
for (long containerID : containerIdList) {
|
||||||
Pipeline pipeline =
|
Pipeline pipeline =
|
||||||
cluster.getStorageContainerManager().getContainerManager()
|
cluster.getStorageContainerManager().getContainerManager()
|
||||||
.getContainerWithPipeline(containerID).getPipeline();
|
.getContainerWithPipeline(ContainerID.valueof(containerID))
|
||||||
|
.getPipeline();
|
||||||
pipelineList.add(pipeline);
|
pipelineList.add(pipeline);
|
||||||
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
||||||
for (DatanodeDetails details : datanodes) {
|
for (DatanodeDetails details : datanodes) {
|
||||||
|
@ -349,7 +351,8 @@ public class TestCloseContainerHandlingByClient {
|
||||||
long containerID = locationInfos.get(0).getContainerID();
|
long containerID = locationInfos.get(0).getContainerID();
|
||||||
List<DatanodeDetails> datanodes =
|
List<DatanodeDetails> datanodes =
|
||||||
cluster.getStorageContainerManager().getContainerManager()
|
cluster.getStorageContainerManager().getContainerManager()
|
||||||
.getContainerWithPipeline(containerID).getPipeline().getMachines();
|
.getContainerWithPipeline(ContainerID.valueof(containerID))
|
||||||
|
.getPipeline().getMachines();
|
||||||
Assert.assertEquals(1, datanodes.size());
|
Assert.assertEquals(1, datanodes.size());
|
||||||
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE);
|
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE);
|
||||||
dataString = fixedLengthString(keyString, (1 * blockSize));
|
dataString = fixedLengthString(keyString, (1 * blockSize));
|
||||||
|
@ -451,7 +454,8 @@ public class TestCloseContainerHandlingByClient {
|
||||||
long containerID = locationInfos.get(0).getContainerID();
|
long containerID = locationInfos.get(0).getContainerID();
|
||||||
List<DatanodeDetails> datanodes =
|
List<DatanodeDetails> datanodes =
|
||||||
cluster.getStorageContainerManager().getContainerManager()
|
cluster.getStorageContainerManager().getContainerManager()
|
||||||
.getContainerWithPipeline(containerID).getPipeline().getMachines();
|
.getContainerWithPipeline(ContainerID.valueof(containerID))
|
||||||
|
.getPipeline().getMachines();
|
||||||
Assert.assertEquals(1, datanodes.size());
|
Assert.assertEquals(1, datanodes.size());
|
||||||
// move the container on the datanode to Closing state, this will ensure
|
// move the container on the datanode to Closing state, this will ensure
|
||||||
// closing the key will hit BLOCK_NOT_COMMITTED_EXCEPTION while trying
|
// closing the key will hit BLOCK_NOT_COMMITTED_EXCEPTION while trying
|
||||||
|
|
|
@ -23,7 +23,8 @@ import org.apache.commons.lang3.RandomUtils;
|
||||||
import org.apache.hadoop.hdds.protocol.StorageType;
|
import org.apache.hadoop.hdds.protocol.StorageType;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.ozone.*;
|
import org.apache.hadoop.ozone.*;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
|
@ -642,7 +643,8 @@ public class TestOzoneRpcClient {
|
||||||
// Second, sum the data size from chunks in Container via containerID
|
// Second, sum the data size from chunks in Container via containerID
|
||||||
// and localID, make sure the size equals to the size from keyDetails.
|
// and localID, make sure the size equals to the size from keyDetails.
|
||||||
Pipeline pipeline = cluster.getStorageContainerManager()
|
Pipeline pipeline = cluster.getStorageContainerManager()
|
||||||
.getContainerManager().getContainerWithPipeline(containerID)
|
.getContainerManager().getContainerWithPipeline(
|
||||||
|
ContainerID.valueof(containerID))
|
||||||
.getPipeline();
|
.getPipeline();
|
||||||
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
||||||
Assert.assertEquals(datanodes.size(), 1);
|
Assert.assertEquals(datanodes.size(), 1);
|
||||||
|
|
|
@ -242,8 +242,7 @@ public class TestBlockDeletion {
|
||||||
|
|
||||||
logCapturer.clearOutput();
|
logCapturer.clearOutput();
|
||||||
scm.getContainerManager().processContainerReports(
|
scm.getContainerManager().processContainerReports(
|
||||||
cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport,
|
cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport);
|
||||||
false);
|
|
||||||
// wait for event to be handled by event handler
|
// wait for event to be handled by event handler
|
||||||
Thread.sleep(1000);
|
Thread.sleep(1000);
|
||||||
String output = logCapturer.getOutput();
|
String output = logCapturer.getOutput();
|
||||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.ozone.HddsDatanodeService;
|
import org.apache.hadoop.ozone.HddsDatanodeService;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
|
@ -102,7 +103,8 @@ public class TestCloseContainerByPipeline {
|
||||||
|
|
||||||
long containerID = omKeyLocationInfo.getContainerID();
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
Pipeline pipeline = cluster.getStorageContainerManager()
|
Pipeline pipeline = cluster.getStorageContainerManager()
|
||||||
.getContainerManager().getContainerWithPipeline(containerID)
|
.getContainerManager().getContainerWithPipeline(
|
||||||
|
ContainerID.valueof(containerID))
|
||||||
.getPipeline();
|
.getPipeline();
|
||||||
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
||||||
Assert.assertEquals(datanodes.size(), 1);
|
Assert.assertEquals(datanodes.size(), 1);
|
||||||
|
@ -157,7 +159,8 @@ public class TestCloseContainerByPipeline {
|
||||||
|
|
||||||
long containerID = omKeyLocationInfo.getContainerID();
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
Pipeline pipeline = cluster.getStorageContainerManager()
|
Pipeline pipeline = cluster.getStorageContainerManager()
|
||||||
.getContainerManager().getContainerWithPipeline(containerID)
|
.getContainerManager().getContainerWithPipeline(
|
||||||
|
ContainerID.valueof(containerID))
|
||||||
.getPipeline();
|
.getPipeline();
|
||||||
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
||||||
Assert.assertEquals(datanodes.size(), 1);
|
Assert.assertEquals(datanodes.size(), 1);
|
||||||
|
@ -214,7 +217,8 @@ public class TestCloseContainerByPipeline {
|
||||||
|
|
||||||
long containerID = omKeyLocationInfo.getContainerID();
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
Pipeline pipeline = cluster.getStorageContainerManager()
|
Pipeline pipeline = cluster.getStorageContainerManager()
|
||||||
.getContainerManager().getContainerWithPipeline(containerID)
|
.getContainerManager().getContainerWithPipeline(
|
||||||
|
ContainerID.valueof(containerID))
|
||||||
.getPipeline();
|
.getPipeline();
|
||||||
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
List<DatanodeDetails> datanodes = pipeline.getMachines();
|
||||||
Assert.assertEquals(3, datanodes.size());
|
Assert.assertEquals(3, datanodes.size());
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
import org.apache.hadoop.ozone.client.ObjectStore;
|
import org.apache.hadoop.ozone.client.ObjectStore;
|
||||||
|
@ -81,7 +82,8 @@ public class TestCloseContainerHandler {
|
||||||
|
|
||||||
long containerID = omKeyLocationInfo.getContainerID();
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
Pipeline pipeline = cluster.getStorageContainerManager()
|
Pipeline pipeline = cluster.getStorageContainerManager()
|
||||||
.getContainerManager().getContainerWithPipeline(containerID)
|
.getContainerManager().getContainerWithPipeline(
|
||||||
|
ContainerID.valueof(containerID))
|
||||||
.getPipeline();
|
.getPipeline();
|
||||||
|
|
||||||
Assert.assertFalse(isContainerClosed(cluster, containerID));
|
Assert.assertFalse(isContainerClosed(cluster, containerID));
|
||||||
|
|
|
@ -22,7 +22,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor;
|
||||||
import org.apache.hadoop.hdds.client.ReplicationType;
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
import org.apache.hadoop.ozone.client.*;
|
import org.apache.hadoop.ozone.client.*;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
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.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
|
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||||
|
@ -127,8 +127,7 @@ public class TestScmChillMode {
|
||||||
new TestStorageContainerManagerHelper(cluster, conf);
|
new TestStorageContainerManagerHelper(cluster, conf);
|
||||||
Map<String, OmKeyInfo> keyLocations = helper.createKeys(100, 4096);
|
Map<String, OmKeyInfo> keyLocations = helper.createKeys(100, 4096);
|
||||||
final List<ContainerInfo> containers = cluster
|
final List<ContainerInfo> containers = cluster
|
||||||
.getStorageContainerManager()
|
.getStorageContainerManager().getContainerManager().getContainers();
|
||||||
.getContainerManager().getStateManager().getAllContainers();
|
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return containers.size() > 10;
|
return containers.size() > 10;
|
||||||
}, 100, 1000);
|
}, 100, 1000);
|
||||||
|
@ -251,8 +250,7 @@ public class TestScmChillMode {
|
||||||
new TestStorageContainerManagerHelper(miniCluster, conf);
|
new TestStorageContainerManagerHelper(miniCluster, conf);
|
||||||
Map<String, OmKeyInfo> keyLocations = helper.createKeys(100 * 2, 4096);
|
Map<String, OmKeyInfo> keyLocations = helper.createKeys(100 * 2, 4096);
|
||||||
final List<ContainerInfo> containers = miniCluster
|
final List<ContainerInfo> containers = miniCluster
|
||||||
.getStorageContainerManager().getContainerManager()
|
.getStorageContainerManager().getContainerManager().getContainers();
|
||||||
.getStateManager().getAllContainers();
|
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return containers.size() > 10;
|
return containers.size() > 10;
|
||||||
}, 100, 1000 * 2);
|
}, 100, 1000 * 2);
|
||||||
|
@ -268,9 +266,9 @@ public class TestScmChillMode {
|
||||||
.getStorageContainerManager().getContainerManager();
|
.getStorageContainerManager().getContainerManager();
|
||||||
containers.forEach(c -> {
|
containers.forEach(c -> {
|
||||||
try {
|
try {
|
||||||
mapping.updateContainerState(c.getContainerID(),
|
mapping.updateContainerState(c.containerID(),
|
||||||
HddsProtos.LifeCycleEvent.FINALIZE);
|
HddsProtos.LifeCycleEvent.FINALIZE);
|
||||||
mapping.updateContainerState(c.getContainerID(),
|
mapping.updateContainerState(c.containerID(),
|
||||||
LifeCycleEvent.CLOSE);
|
LifeCycleEvent.CLOSE);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.info("Failed to change state of open containers.", e);
|
LOG.info("Failed to change state of open containers.", e);
|
||||||
|
@ -348,7 +346,7 @@ public class TestScmChillMode {
|
||||||
.getStorageContainerManager().getClientProtocolServer();
|
.getStorageContainerManager().getClientProtocolServer();
|
||||||
assertFalse((scm.getClientProtocolServer()).getChillModeStatus());
|
assertFalse((scm.getClientProtocolServer()).getChillModeStatus());
|
||||||
final List<ContainerInfo> containers = scm.getContainerManager()
|
final List<ContainerInfo> containers = scm.getContainerManager()
|
||||||
.getStateManager().getAllContainers();
|
.getContainers();
|
||||||
scm.getEventQueue().fireEvent(SCMEvents.CHILL_MODE_STATUS, true);
|
scm.getEventQueue().fireEvent(SCMEvents.CHILL_MODE_STATUS, true);
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return clientProtocolServer.getChillModeStatus();
|
return clientProtocolServer.getChillModeStatus();
|
||||||
|
|
|
@ -411,10 +411,6 @@ public class KeyManagerImpl implements KeyManager {
|
||||||
|
|
||||||
// A rename is a no-op if the target and source name is same.
|
// A rename is a no-op if the target and source name is same.
|
||||||
// TODO: Discuss if we need to throw?.
|
// TODO: Discuss if we need to throw?.
|
||||||
// TODO: Define the semantics of rename more clearly. Today this code
|
|
||||||
// will allow rename of a Key across volumes. This should *not* be
|
|
||||||
// allowed. The documentation of Ozone says that rename is permitted only
|
|
||||||
// within a volume.
|
|
||||||
if (fromKeyName.equals(toKeyName)) {
|
if (fromKeyName.equals(toKeyName)) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,7 @@ import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
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.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
||||||
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
||||||
|
@ -70,9 +70,6 @@ public class BenchMarkContainerStateMap {
|
||||||
.setPipelineID(pipeline.getId())
|
.setPipelineID(pipeline.getId())
|
||||||
.setReplicationType(pipeline.getType())
|
.setReplicationType(pipeline.getType())
|
||||||
.setReplicationFactor(pipeline.getFactor())
|
.setReplicationFactor(pipeline.getFactor())
|
||||||
// This is bytes allocated for blocks inside container, not the
|
|
||||||
// container size
|
|
||||||
.setAllocatedBytes(0)
|
|
||||||
.setUsedBytes(0)
|
.setUsedBytes(0)
|
||||||
.setNumberOfKeys(0)
|
.setNumberOfKeys(0)
|
||||||
.setStateEnterTime(Time.monotonicNow())
|
.setStateEnterTime(Time.monotonicNow())
|
||||||
|
@ -93,9 +90,6 @@ public class BenchMarkContainerStateMap {
|
||||||
.setPipelineID(pipeline.getId())
|
.setPipelineID(pipeline.getId())
|
||||||
.setReplicationType(pipeline.getType())
|
.setReplicationType(pipeline.getType())
|
||||||
.setReplicationFactor(pipeline.getFactor())
|
.setReplicationFactor(pipeline.getFactor())
|
||||||
// This is bytes allocated for blocks inside container, not the
|
|
||||||
// container size
|
|
||||||
.setAllocatedBytes(0)
|
|
||||||
.setUsedBytes(0)
|
.setUsedBytes(0)
|
||||||
.setNumberOfKeys(0)
|
.setNumberOfKeys(0)
|
||||||
.setStateEnterTime(Time.monotonicNow())
|
.setStateEnterTime(Time.monotonicNow())
|
||||||
|
@ -115,9 +109,6 @@ public class BenchMarkContainerStateMap {
|
||||||
.setPipelineID(pipeline.getId())
|
.setPipelineID(pipeline.getId())
|
||||||
.setReplicationType(pipeline.getType())
|
.setReplicationType(pipeline.getType())
|
||||||
.setReplicationFactor(pipeline.getFactor())
|
.setReplicationFactor(pipeline.getFactor())
|
||||||
// This is bytes allocated for blocks inside container, not the
|
|
||||||
// container size
|
|
||||||
.setAllocatedBytes(0)
|
|
||||||
.setUsedBytes(0)
|
.setUsedBytes(0)
|
||||||
.setNumberOfKeys(0)
|
.setNumberOfKeys(0)
|
||||||
.setStateEnterTime(Time.monotonicNow())
|
.setStateEnterTime(Time.monotonicNow())
|
||||||
|
@ -188,9 +179,6 @@ public class BenchMarkContainerStateMap {
|
||||||
.setPipelineID(pipeline.getId())
|
.setPipelineID(pipeline.getId())
|
||||||
.setReplicationType(pipeline.getType())
|
.setReplicationType(pipeline.getType())
|
||||||
.setReplicationFactor(pipeline.getFactor())
|
.setReplicationFactor(pipeline.getFactor())
|
||||||
// This is bytes allocated for blocks inside container, not the
|
|
||||||
// container size
|
|
||||||
.setAllocatedBytes(0)
|
|
||||||
.setUsedBytes(0)
|
.setUsedBytes(0)
|
||||||
.setNumberOfKeys(0)
|
.setNumberOfKeys(0)
|
||||||
.setStateEnterTime(Time.monotonicNow())
|
.setStateEnterTime(Time.monotonicNow())
|
||||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo
|
||||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.util.Tool;
|
import org.apache.hadoop.util.Tool;
|
||||||
import org.apache.hadoop.util.ToolRunner;
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
import org.apache.hadoop.utils.MetadataStore;
|
import org.apache.hadoop.utils.MetadataStore;
|
||||||
|
@ -83,7 +83,6 @@ public class SQLCLI extends Configured implements Tool {
|
||||||
"replicationType TEXT NOT NULL," +
|
"replicationType TEXT NOT NULL," +
|
||||||
"replicationFactor TEXT NOT NULL," +
|
"replicationFactor TEXT NOT NULL," +
|
||||||
"usedBytes LONG NOT NULL," +
|
"usedBytes LONG NOT NULL," +
|
||||||
"allocatedBytes LONG NOT NULL," +
|
|
||||||
"owner TEXT," +
|
"owner TEXT," +
|
||||||
"numberOfKeys LONG)";
|
"numberOfKeys LONG)";
|
||||||
private static final String CREATE_DATANODE_INFO =
|
private static final String CREATE_DATANODE_INFO =
|
||||||
|
@ -94,8 +93,8 @@ public class SQLCLI extends Configured implements Tool {
|
||||||
"containerPort INTEGER NOT NULL);";
|
"containerPort INTEGER NOT NULL);";
|
||||||
private static final String INSERT_CONTAINER_INFO =
|
private static final String INSERT_CONTAINER_INFO =
|
||||||
"INSERT INTO containerInfo (containerID, replicationType, "
|
"INSERT INTO containerInfo (containerID, replicationType, "
|
||||||
+ "replicationFactor, usedBytes, allocatedBytes, owner, "
|
+ "replicationFactor, usedBytes, owner, "
|
||||||
+ "numberOfKeys) VALUES (\"%d\", \"%s\", \"%s\", \"%d\", \"%d\", "
|
+ "numberOfKeys) VALUES (\"%d\", \"%s\", \"%s\", \"%d\", "
|
||||||
+ "\"%s\", \"%d\")";
|
+ "\"%s\", \"%d\")";
|
||||||
private static final String INSERT_DATANODE_INFO =
|
private static final String INSERT_DATANODE_INFO =
|
||||||
"INSERT INTO datanodeInfo (hostname, datanodeUUid, ipAddress, " +
|
"INSERT INTO datanodeInfo (hostname, datanodeUUid, ipAddress, " +
|
||||||
|
@ -498,7 +497,6 @@ public class SQLCLI extends Configured implements Tool {
|
||||||
containerInfo.getReplicationType(),
|
containerInfo.getReplicationType(),
|
||||||
containerInfo.getReplicationFactor(),
|
containerInfo.getReplicationFactor(),
|
||||||
containerInfo.getUsedBytes(),
|
containerInfo.getUsedBytes(),
|
||||||
containerInfo.getAllocatedBytes(),
|
|
||||||
containerInfo.getOwner(),
|
containerInfo.getOwner(),
|
||||||
containerInfo.getNumberOfKeys());
|
containerInfo.getNumberOfKeys());
|
||||||
|
|
||||||
|
|
|
@ -120,8 +120,7 @@ public class TestContainerSQLCli {
|
||||||
cluster.getStorageContainerManager().stop();
|
cluster.getStorageContainerManager().stop();
|
||||||
eventQueue = new EventQueue();
|
eventQueue = new EventQueue();
|
||||||
nodeManager = cluster.getStorageContainerManager().getScmNodeManager();
|
nodeManager = cluster.getStorageContainerManager().getScmNodeManager();
|
||||||
containerManager = new SCMContainerManager(conf, nodeManager, 128,
|
containerManager = new SCMContainerManager(conf, nodeManager, eventQueue);
|
||||||
eventQueue);
|
|
||||||
blockManager = new BlockManagerImpl(
|
blockManager = new BlockManagerImpl(
|
||||||
conf, nodeManager, containerManager, eventQueue);
|
conf, nodeManager, containerManager, eventQueue);
|
||||||
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);
|
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);
|
||||||
|
|
Loading…
Reference in New Issue