HDDS-755. ContainerInfo and ContainerReplica protobuf changes.

Contributed by Nanda kumar.
This commit is contained in:
Nanda kumar 2018-10-31 10:29:35 +05:30
parent 773f0d1519
commit e4f22b08e0
33 changed files with 199 additions and 185 deletions

View File

@ -27,7 +27,7 @@ import org.apache.hadoop.hdds.scm.protocolPB
.StorageContainerLocationProtocolClientSideTranslatorPB; .StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerData; .ContainerDataProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ReadContainerResponseProto; .ReadContainerResponseProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@ -309,7 +309,7 @@ public class ContainerOperationClient implements ScmClient {
* @throws IOException * @throws IOException
*/ */
@Override @Override
public ContainerData readContainer(long containerID, public ContainerDataProto readContainer(long containerID,
Pipeline pipeline) throws IOException { Pipeline pipeline) throws IOException {
XceiverClientSpi client = null; XceiverClientSpi client = null;
try { try {
@ -337,7 +337,7 @@ public class ContainerOperationClient implements ScmClient {
* @throws IOException * @throws IOException
*/ */
@Override @Override
public ContainerData readContainer(long containerID) throws IOException { public ContainerDataProto readContainer(long containerID) throws IOException {
ContainerWithPipeline info = getContainerWithPipeline(containerID); ContainerWithPipeline info = getContainerWithPipeline(containerID);
return readContainer(containerID, info.getPipeline()); return readContainer(containerID, info.getPipeline());
} }

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline
import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerData; .ContainerDataProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import java.io.Closeable; import java.io.Closeable;
@ -119,7 +119,7 @@ public interface ScmClient extends Closeable {
* @return ContainerInfo * @return ContainerInfo
* @throws IOException * @throws IOException
*/ */
ContainerData readContainer(long containerID, Pipeline pipeline) ContainerDataProto readContainer(long containerID, Pipeline pipeline)
throws IOException; throws IOException;
/** /**
@ -128,7 +128,7 @@ public interface ScmClient extends Closeable {
* @return ContainerInfo * @return ContainerInfo
* @throws IOException * @throws IOException
*/ */
ContainerData readContainer(long containerID) ContainerDataProto readContainer(long containerID)
throws IOException; throws IOException;
/** /**

View File

@ -114,7 +114,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
public ContainerInfo() { public ContainerInfo() {
} }
public static ContainerInfo fromProtobuf(HddsProtos.SCMContainerInfo info) { public static ContainerInfo fromProtobuf(HddsProtos.ContainerInfoProto info) {
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()))
@ -191,9 +191,9 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
lastUsed = Time.monotonicNow(); lastUsed = Time.monotonicNow();
} }
public HddsProtos.SCMContainerInfo getProtobuf() { public HddsProtos.ContainerInfoProto getProtobuf() {
HddsProtos.SCMContainerInfo.Builder builder = HddsProtos.ContainerInfoProto.Builder builder =
HddsProtos.SCMContainerInfo.newBuilder(); HddsProtos.ContainerInfoProto.newBuilder();
Preconditions.checkState(containerID > 0); Preconditions.checkState(containerID > 0);
return builder.setContainerID(getContainerID()) return builder.setContainerID(getContainerID())
.setUsedBytes(getUsedBytes()) .setUsedBytes(getUsedBytes())

View File

@ -181,7 +181,7 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
SCMListContainerResponseProto response = SCMListContainerResponseProto response =
rpcProxy.listContainer(NULL_RPC_CONTROLLER, request); rpcProxy.listContainer(NULL_RPC_CONTROLLER, request);
List<ContainerInfo> containerList = new ArrayList<>(); List<ContainerInfo> containerList = new ArrayList<>();
for (HddsProtos.SCMContainerInfo containerInfoProto : response for (HddsProtos.ContainerInfoProto containerInfoProto : response
.getContainersList()) { .getContainersList()) {
containerList.add(ContainerInfo.fromProtobuf(containerInfoProto)); containerList.add(ContainerInfo.fromProtobuf(containerInfoProto));
} }

View File

@ -158,17 +158,6 @@ message KeyValue {
optional string value = 2; optional string value = 2;
} }
/**
* Lifecycle states of a container in Datanode.
*/
enum ContainerLifeCycleState {
OPEN = 1;
CLOSING = 2;
CLOSED = 3;
UNHEALTHY = 4;
INVALID = 5;
}
message ContainerCommandRequestProto { message ContainerCommandRequestProto {
required Type cmdType = 1; // Type of the command required Type cmdType = 1; // Type of the command
@ -235,14 +224,22 @@ message ContainerCommandResponseProto {
optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21; optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21;
} }
message ContainerData { message ContainerDataProto {
enum State {
OPEN = 1;
CLOSING = 2;
CLOSED = 3;
QUASI_CLOSED = 4;
UNHEALTHY = 5;
INVALID = 6;
}
required int64 containerID = 1; required int64 containerID = 1;
repeated KeyValue metadata = 2; repeated KeyValue metadata = 2;
optional string containerPath = 4; optional string containerPath = 4;
optional int64 bytesUsed = 6; optional int64 bytesUsed = 6;
optional int64 size = 7; optional int64 size = 7;
optional int64 blockCount = 8; optional int64 blockCount = 8;
optional ContainerLifeCycleState state = 9 [default = OPEN]; optional State state = 9 [default = OPEN];
optional ContainerType containerType = 10 [default = KeyValueContainer]; optional ContainerType containerType = 10 [default = KeyValueContainer];
} }
@ -264,7 +261,7 @@ message ReadContainerRequestProto {
} }
message ReadContainerResponseProto { message ReadContainerResponseProto {
optional ContainerData containerData = 1; optional ContainerDataProto containerData = 1;
} }
message UpdateContainerRequestProto { message UpdateContainerRequestProto {
@ -287,7 +284,7 @@ message ListContainerRequestProto {
} }
message ListContainerResponseProto { message ListContainerResponseProto {
repeated ContainerData containerData = 1; repeated ContainerDataProto containerData = 1;
} }
message CloseContainerRequestProto { message CloseContainerRequestProto {

View File

@ -60,7 +60,7 @@ message GetContainerRequestProto {
} }
message GetContainerResponseProto { message GetContainerResponseProto {
required SCMContainerInfo containerInfo = 1; required ContainerInfoProto containerInfo = 1;
} }
message GetContainerWithPipelineRequestProto { message GetContainerWithPipelineRequestProto {
@ -77,7 +77,7 @@ message SCMListContainerRequestProto {
} }
message SCMListContainerResponseProto { message SCMListContainerResponseProto {
repeated SCMContainerInfo containers = 1; repeated ContainerInfoProto containers = 1;
} }
message SCMDeleteContainerRequestProto { message SCMDeleteContainerRequestProto {

View File

@ -131,7 +131,7 @@ enum LifeCycleEvent {
CLEANUP = 8; CLEANUP = 8;
} }
message SCMContainerInfo { message ContainerInfoProto {
required int64 containerID = 1; required int64 containerID = 1;
required LifeCycleState state = 2; required LifeCycleState state = 2;
optional PipelineID pipelineID = 3; optional PipelineID pipelineID = 3;
@ -145,7 +145,7 @@ message SCMContainerInfo {
} }
message ContainerWithPipeline { message ContainerWithPipeline {
required SCMContainerInfo containerInfo = 1; required ContainerInfoProto containerInfo = 1;
required Pipeline pipeline = 2; required Pipeline pipeline = 2;
} }

View File

@ -25,8 +25,8 @@ import java.util.List;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos. import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
ContainerType; ContainerType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos. import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
ContainerLifeCycleState; .ContainerDataProto;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
@ -65,7 +65,7 @@ public abstract class ContainerData {
private final Map<String, String> metadata; private final Map<String, String> metadata;
// State of the Container // State of the Container
private ContainerLifeCycleState state; private ContainerDataProto.State state;
private final long maxSize; private final long maxSize;
@ -121,7 +121,7 @@ public abstract class ContainerData {
this.containerID = containerId; this.containerID = containerId;
this.layOutVersion = layOutVersion; this.layOutVersion = layOutVersion;
this.metadata = new TreeMap<>(); this.metadata = new TreeMap<>();
this.state = ContainerLifeCycleState.OPEN; this.state = ContainerDataProto.State.OPEN;
this.readCount = new AtomicLong(0L); this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L); this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L); this.writeCount = new AtomicLong(0L);
@ -158,7 +158,7 @@ public abstract class ContainerData {
* Returns the state of the container. * Returns the state of the container.
* @return ContainerLifeCycleState * @return ContainerLifeCycleState
*/ */
public synchronized ContainerLifeCycleState getState() { public synchronized ContainerDataProto.State getState() {
return state; return state;
} }
@ -166,7 +166,7 @@ public abstract class ContainerData {
* Set the state of the container. * Set the state of the container.
* @param state * @param state
*/ */
public synchronized void setState(ContainerLifeCycleState state) { public synchronized void setState(ContainerDataProto.State state) {
this.state = state; this.state = state;
} }
@ -222,7 +222,7 @@ public abstract class ContainerData {
* @return - boolean * @return - boolean
*/ */
public synchronized boolean isOpen() { public synchronized boolean isOpen() {
return ContainerLifeCycleState.OPEN == state; return ContainerDataProto.State.OPEN == state;
} }
/** /**
@ -230,7 +230,7 @@ public abstract class ContainerData {
* @return - boolean * @return - boolean
*/ */
public synchronized boolean isValid() { public synchronized boolean isValid() {
return !(ContainerLifeCycleState.INVALID == state); return !(ContainerDataProto.State.INVALID == state);
} }
/** /**
@ -238,14 +238,14 @@ public abstract class ContainerData {
* @return - boolean * @return - boolean
*/ */
public synchronized boolean isClosed() { public synchronized boolean isClosed() {
return ContainerLifeCycleState.CLOSED == state; return ContainerDataProto.State.CLOSED == state;
} }
/** /**
* Marks this container as closed. * Marks this container as closed.
*/ */
public synchronized void closeContainer() { public synchronized void closeContainer() {
setState(ContainerLifeCycleState.CLOSED); setState(ContainerDataProto.State.CLOSED);
} }
/** /**
@ -431,5 +431,5 @@ public abstract class ContainerData {
* *
* @return Protocol Buffer Message * @return Protocol Buffer Message
*/ */
public abstract ContainerProtos.ContainerData getProtoBufMessage(); public abstract ContainerProtos.ContainerDataProto getProtoBufMessage();
} }

View File

@ -253,13 +253,13 @@ public final class ContainerDataYaml {
String state = (String) nodes.get(OzoneConsts.STATE); String state = (String) nodes.get(OzoneConsts.STATE);
switch (state) { switch (state) {
case "OPEN": case "OPEN":
kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN); kvData.setState(ContainerProtos.ContainerDataProto.State.OPEN);
break; break;
case "CLOSING": case "CLOSING":
kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING); kvData.setState(ContainerProtos.ContainerDataProto.State.CLOSING);
break; break;
case "CLOSED": case "CLOSED":
kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED); kvData.setState(ContainerProtos.ContainerDataProto.State.CLOSED);
break; break;
default: default:
throw new IllegalStateException("Unexpected " + throw new IllegalStateException("Unexpected " +

View File

@ -24,6 +24,8 @@ import com.google.common.collect.Maps;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerDataProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerAction; .StorageContainerDatanodeProtocolProtos.ContainerAction;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
@ -40,8 +42,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto; .ContainerCommandResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerType; .ContainerType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerLifeCycleState;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -156,9 +156,9 @@ public class HddsDispatcher implements ContainerDispatcher {
// which has failed, so the container is marked unhealthy right here. // which has failed, so the container is marked unhealthy right here.
// Once container is marked unhealthy, all the subsequent write // Once container is marked unhealthy, all the subsequent write
// transactions will fail with UNHEALTHY_CONTAINER exception. // transactions will fail with UNHEALTHY_CONTAINER exception.
if (container.getContainerState() == ContainerLifeCycleState.OPEN) { if (container.getContainerState() == ContainerDataProto.State.OPEN) {
container.getContainerData() container.getContainerData()
.setState(ContainerLifeCycleState.UNHEALTHY); .setState(ContainerDataProto.State.UNHEALTHY);
sendCloseContainerActionIfNeeded(container); sendCloseContainerActionIfNeeded(container);
} }
} }
@ -191,7 +191,7 @@ public class HddsDispatcher implements ContainerDispatcher {
private boolean isContainerFull(Container container) { private boolean isContainerFull(Container container) {
boolean isOpen = Optional.ofNullable(container) boolean isOpen = Optional.ofNullable(container)
.map(cont -> cont.getContainerState() == ContainerLifeCycleState.OPEN) .map(cont -> cont.getContainerState() == ContainerDataProto.State.OPEN)
.orElse(Boolean.FALSE); .orElse(Boolean.FALSE);
if (isOpen) { if (isOpen) {
ContainerData containerData = container.getContainerData(); ContainerData containerData = container.getContainerData();
@ -205,7 +205,8 @@ public class HddsDispatcher implements ContainerDispatcher {
private boolean isContainerUnhealthy(Container container) { private boolean isContainerUnhealthy(Container container) {
return Optional.ofNullable(container).map( return Optional.ofNullable(container).map(
cont -> (cont.getContainerState() == ContainerLifeCycleState.UNHEALTHY)) cont -> (cont.getContainerState() ==
ContainerDataProto.State.UNHEALTHY))
.orElse(Boolean.FALSE); .orElse(Boolean.FALSE);
} }

View File

@ -25,11 +25,10 @@ import java.io.OutputStream;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.proto
.ContainerLifeCycleState; .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.scm.container.common.helpers import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException; .StorageContainerException;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdfs.util.RwLock; import org.apache.hadoop.hdfs.util.RwLock;
import org.apache.hadoop.ozone.container.common.impl.ContainerData; import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@ -80,7 +79,7 @@ public interface Container<CONTAINERDATA extends ContainerData> extends RwLock {
* @return ContainerLifeCycleState - Container State. * @return ContainerLifeCycleState - Container State.
* @throws StorageContainerException * @throws StorageContainerException
*/ */
ContainerLifeCycleState getContainerState(); ContainerProtos.ContainerDataProto.State getContainerState();
/** /**
* Closes a open container, if it is already closed or does not exist a * Closes a open container, if it is already closed or does not exist a
@ -130,7 +129,7 @@ public interface Container<CONTAINERDATA extends ContainerData> extends RwLock {
/** /**
* Returns containerReport for the container. * Returns containerReport for the container.
*/ */
StorageContainerDatanodeProtocolProtos.ContainerInfo getContainerReport() ContainerReplicaProto getContainerReport()
throws StorageContainerException; throws StorageContainerException;
/** /**

View File

@ -30,13 +30,11 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
.ContainerLifeCycleState;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerType; .ContainerType;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.scm.container.common.helpers import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException; .StorageContainerException;
import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.io.nativeio.NativeIO;
@ -284,7 +282,7 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
} catch (StorageContainerException ex) { } catch (StorageContainerException ex) {
// Failed to update .container file. Reset the state to CLOSING // Failed to update .container file. Reset the state to CLOSING
containerData.setState(ContainerLifeCycleState.CLOSING); containerData.setState(ContainerProtos.ContainerDataProto.State.CLOSING);
throw ex; throw ex;
} finally { } finally {
writeUnlock(); writeUnlock();
@ -309,7 +307,7 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
} }
@Override @Override
public ContainerLifeCycleState getContainerState() { public ContainerProtos.ContainerDataProto.State getContainerState() {
return containerData.getState(); return containerData.getState();
} }
@ -427,7 +425,8 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
@Override @Override
public void exportContainerData(OutputStream destination, public void exportContainerData(OutputStream destination,
ContainerPacker<KeyValueContainerData> packer) throws IOException { ContainerPacker<KeyValueContainerData> packer) throws IOException {
if (getContainerData().getState() != ContainerLifeCycleState.CLOSED) { if (getContainerData().getState() !=
ContainerProtos.ContainerDataProto.State.CLOSED) {
throw new IllegalStateException( throw new IllegalStateException(
"Only closed containers could be exported: ContainerId=" "Only closed containers could be exported: ContainerId="
+ getContainerData().getContainerID()); + getContainerData().getContainerID());
@ -518,10 +517,10 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
* Returns KeyValueContainerReport for the KeyValueContainer. * Returns KeyValueContainerReport for the KeyValueContainer.
*/ */
@Override @Override
public StorageContainerDatanodeProtocolProtos.ContainerInfo public ContainerReplicaProto getContainerReport()
getContainerReport() throws StorageContainerException{ throws StorageContainerException {
StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = ContainerReplicaProto.Builder ciBuilder =
StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); ContainerReplicaProto.newBuilder();
ciBuilder.setContainerID(containerData.getContainerID()) ciBuilder.setContainerID(containerData.getContainerID())
.setReadCount(containerData.getReadCount()) .setReadCount(containerData.getReadCount())
.setWriteCount(containerData.getWriteCount()) .setWriteCount(containerData.getWriteCount())
@ -540,18 +539,18 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
* @return LifeCycle State of the container in HddsProtos format * @return LifeCycle State of the container in HddsProtos format
* @throws StorageContainerException * @throws StorageContainerException
*/ */
private HddsProtos.LifeCycleState getHddsState() private ContainerReplicaProto.State getHddsState()
throws StorageContainerException { throws StorageContainerException {
HddsProtos.LifeCycleState state; ContainerReplicaProto.State state;
switch (containerData.getState()) { switch (containerData.getState()) {
case OPEN: case OPEN:
state = HddsProtos.LifeCycleState.OPEN; state = ContainerReplicaProto.State.OPEN;
break; break;
case CLOSING: case CLOSING:
state = HddsProtos.LifeCycleState.CLOSING; state = ContainerReplicaProto.State.CLOSING;
break; break;
case CLOSED: case CLOSED:
state = HddsProtos.LifeCycleState.CLOSED; state = ContainerReplicaProto.State.CLOSED;
break; break;
default: default:
throw new StorageContainerException("Invalid Container state found: " + throw new StorageContainerException("Invalid Container state found: " +

View File

@ -24,6 +24,8 @@ import java.util.Collections;
import org.apache.hadoop.conf.StorageSize; import org.apache.hadoop.conf.StorageSize;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerDataProto;
import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.impl.ContainerData; import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@ -245,9 +247,8 @@ public class KeyValueContainerData extends ContainerData {
* *
* @return Protocol Buffer Message * @return Protocol Buffer Message
*/ */
public ContainerProtos.ContainerData getProtoBufMessage() { public ContainerDataProto getProtoBufMessage() {
ContainerProtos.ContainerData.Builder builder = ContainerProtos ContainerDataProto.Builder builder = ContainerDataProto.newBuilder();
.ContainerData.newBuilder();
builder.setContainerID(this.getContainerID()); builder.setContainerID(this.getContainerID());
builder.setContainerPath(this.getMetadataPath()); builder.setContainerPath(this.getMetadataPath());
builder.setState(this.getState()); builder.setState(this.getState());
@ -282,7 +283,7 @@ public class KeyValueContainerData extends ContainerData {
*/ */
@VisibleForTesting @VisibleForTesting
public static KeyValueContainerData getFromProtoBuf( public static KeyValueContainerData getFromProtoBuf(
ContainerProtos.ContainerData protoData) throws IOException { ContainerDataProto protoData) throws IOException {
// TODO: Add containerMaxSize to ContainerProtos.ContainerData // TODO: Add containerMaxSize to ContainerProtos.ContainerData
StorageSize storageSize = StorageSize.parse( StorageSize storageSize = StorageSize.parse(
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT); ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);

View File

@ -31,12 +31,12 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerDataProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandRequestProto; .ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto; .ContainerCommandResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerLifeCycleState;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerType; .ContainerType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@ -385,13 +385,13 @@ public class KeyValueHandler extends Handler {
} }
long containerID = kvContainer.getContainerData().getContainerID(); long containerID = kvContainer.getContainerData().getContainerID();
ContainerLifeCycleState containerState = kvContainer.getContainerState(); ContainerDataProto.State containerState = kvContainer.getContainerState();
try { try {
if (containerState == ContainerLifeCycleState.CLOSED) { if (containerState == ContainerDataProto.State .CLOSED) {
LOG.debug("Container {} is already closed.", containerID); LOG.debug("Container {} is already closed.", containerID);
return ContainerUtils.getSuccessResponse(request); return ContainerUtils.getSuccessResponse(request);
} else if (containerState == ContainerLifeCycleState.INVALID) { } else if (containerState == ContainerDataProto.State .INVALID) {
LOG.debug("Invalid container data. ContainerID: {}", containerID); LOG.debug("Invalid container data. ContainerID: {}", containerID);
throw new StorageContainerException("Invalid container data. " + throw new StorageContainerException("Invalid container data. " +
"ContainerID: " + containerID, INVALID_CONTAINER_STATE); "ContainerID: " + containerID, INVALID_CONTAINER_STATE);
@ -401,7 +401,7 @@ public class KeyValueHandler extends Handler {
// remove the container from open block map once, all the blocks // remove the container from open block map once, all the blocks
// have been committed and the container is closed // have been committed and the container is closed
kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING); kvData.setState(ContainerDataProto.State.CLOSING);
commitPendingBlocks(kvContainer); commitPendingBlocks(kvContainer);
kvContainer.close(); kvContainer.close();
// make sure the the container open keys from BlockMap gets removed // make sure the the container open keys from BlockMap gets removed
@ -798,9 +798,9 @@ public class KeyValueHandler extends Handler {
private void checkContainerOpen(KeyValueContainer kvContainer) private void checkContainerOpen(KeyValueContainer kvContainer)
throws StorageContainerException { throws StorageContainerException {
ContainerLifeCycleState containerState = kvContainer.getContainerState(); ContainerDataProto.State containerState = kvContainer.getContainerState();
if (containerState == ContainerLifeCycleState.OPEN) { if (containerState == ContainerDataProto.State.OPEN) {
return; return;
} else { } else {
String msg = "Requested operation not allowed as ContainerState is " + String msg = "Requested operation not allowed as ContainerState is " +

View File

@ -80,10 +80,11 @@ message SCMHeartbeatRequestProto {
required DatanodeDetailsProto datanodeDetails = 1; required DatanodeDetailsProto datanodeDetails = 1;
optional NodeReportProto nodeReport = 2; optional NodeReportProto nodeReport = 2;
optional ContainerReportsProto containerReport = 3; optional ContainerReportsProto containerReport = 3;
repeated CommandStatusReportsProto commandStatusReports = 4; optional IncrementalContainerReportProto incrementalContainerReport = 4;
optional ContainerActionsProto containerActions = 5; repeated CommandStatusReportsProto commandStatusReports = 5;
optional PipelineActionsProto pipelineActions = 6; optional ContainerActionsProto containerActions = 6;
optional PipelineReportsProto pipelineReports = 7; optional PipelineActionsProto pipelineActions = 7;
optional PipelineReportsProto pipelineReports = 8;
} }
/* /*
@ -128,7 +129,34 @@ enum StorageTypeProto {
} }
message ContainerReportsProto { message ContainerReportsProto {
repeated ContainerInfo reports = 1; repeated ContainerReplicaProto reports = 1;
}
message IncrementalContainerReportProto {
repeated ContainerReplicaProto report = 1;
}
message ContainerReplicaProto {
enum State {
OPEN = 1;
CLOSING = 2;
CLOSED = 3;
QUASI_CLOSED = 4;
UNHEALTHY = 5;
INVALID = 6;
}
required int64 containerID = 1;
required State state = 2;
optional int64 size = 3;
optional int64 used = 4;
optional int64 keyCount = 5;
optional int64 readCount = 6;
optional int64 writeCount = 7;
optional int64 readBytes = 8;
optional int64 writeBytes = 9;
optional string finalhash = 10;
optional int64 deleteTransactionId = 11;
optional uint64 blockCommitSequenceId = 12;
} }
message CommandStatusReportsProto { message CommandStatusReportsProto {
@ -200,25 +228,6 @@ message PipelineAction {
optional ClosePipelineInfo closePipeline = 2; optional ClosePipelineInfo closePipeline = 2;
} }
/**
A container report contains the following information.
*/
message ContainerInfo {
required int64 containerID = 1;
optional int64 size = 2;
optional int64 used = 3;
optional int64 keyCount = 4;
// TODO: move the io count to separate message
optional int64 readCount = 5;
optional int64 writeCount = 6;
optional int64 readBytes = 7;
optional int64 writeBytes = 8;
optional string finalhash = 9;
optional hadoop.hdds.LifeCycleState state = 10;
optional int64 deleteTransactionId = 11;
optional uint64 blockCommitSequenceId = 12;
}
/* /*
* These are commands returned by SCM for to the datanode to execute. * These are commands returned by SCM for to the datanode to execute.
*/ */

View File

@ -18,6 +18,8 @@ package org.apache.hadoop.ozone.container.common;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -31,8 +33,6 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -66,8 +66,9 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
} }
// Map of datanode to containers // Map of datanode to containers
private Map<DatanodeDetails, Map<String, ContainerInfo>> nodeContainers = private Map<DatanodeDetails,
new HashMap(); Map<String, ContainerReplicaProto>> nodeContainers =
new HashMap<>();
private Map<DatanodeDetails, NodeReportProto> nodeReports = new HashMap<>(); private Map<DatanodeDetails, NodeReportProto> nodeReports = new HashMap<>();
private AtomicInteger commandStatusReport = new AtomicInteger(0); private AtomicInteger commandStatusReport = new AtomicInteger(0);
private List<CommandStatus> cmdStatusList = new LinkedList<>(); private List<CommandStatus> cmdStatusList = new LinkedList<>();
@ -274,7 +275,7 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
nodeContainers.put(datanode, containers); nodeContainers.put(datanode, containers);
} }
for (StorageContainerDatanodeProtocolProtos.ContainerInfo report : reports for (ContainerReplicaProto report : reports
.getReportsList()) { .getReportsList()) {
containers.put(report.getContainerID(), report); containers.put(report.getContainerID(), report);
} }
@ -297,7 +298,8 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
* @return count of storage reports of a datanode * @return count of storage reports of a datanode
*/ */
public int getContainerCountsForDatanode(DatanodeDetails datanodeDetails) { public int getContainerCountsForDatanode(DatanodeDetails datanodeDetails) {
Map<String, ContainerInfo> cr = nodeContainers.get(datanodeDetails); Map<String, ContainerReplicaProto> cr =
nodeContainers.get(datanodeDetails);
if(cr != null) { if(cr != null) {
return cr.size(); return cr.size();
} }

View File

@ -40,8 +40,8 @@ public class TestKeyValueContainerData {
.ContainerType.KeyValueContainer; .ContainerType.KeyValueContainer;
String path = "/tmp"; String path = "/tmp";
String containerDBType = "RocksDB"; String containerDBType = "RocksDB";
ContainerProtos.ContainerLifeCycleState state = ContainerProtos ContainerProtos.ContainerDataProto.State state =
.ContainerLifeCycleState.CLOSED; ContainerProtos.ContainerDataProto.State.CLOSED;
AtomicLong val = new AtomicLong(0); AtomicLong val = new AtomicLong(0);
KeyValueContainerData kvData = new KeyValueContainerData(containerId, KeyValueContainerData kvData = new KeyValueContainerData(containerId,
@ -49,7 +49,7 @@ public class TestKeyValueContainerData {
assertEquals(containerType, kvData.getContainerType()); assertEquals(containerType, kvData.getContainerType());
assertEquals(containerId, kvData.getContainerID()); assertEquals(containerId, kvData.getContainerID());
assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData assertEquals(ContainerProtos.ContainerDataProto.State.OPEN, kvData
.getState()); .getState());
assertEquals(0, kvData.getMetadata().size()); assertEquals(0, kvData.getMetadata().size());
assertEquals(0, kvData.getNumPendingDeletionBlocks()); assertEquals(0, kvData.getNumPendingDeletionBlocks());

View File

@ -91,7 +91,7 @@ public class TestContainerDataYaml {
assertEquals("RocksDB", kvData.getContainerDBType()); assertEquals("RocksDB", kvData.getContainerDBType());
assertEquals(containerFile.getParent(), kvData.getMetadataPath()); assertEquals(containerFile.getParent(), kvData.getMetadataPath());
assertEquals(containerFile.getParent(), kvData.getChunksPath()); assertEquals(containerFile.getParent(), kvData.getChunksPath());
assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData assertEquals(ContainerProtos.ContainerDataProto.State.OPEN, kvData
.getState()); .getState());
assertEquals(1, kvData.getLayOutVersion()); assertEquals(1, kvData.getLayOutVersion());
assertEquals(0, kvData.getMetadata().size()); assertEquals(0, kvData.getMetadata().size());
@ -100,7 +100,7 @@ public class TestContainerDataYaml {
// Update ContainerData. // Update ContainerData.
kvData.addMetadata("VOLUME", "hdfs"); kvData.addMetadata("VOLUME", "hdfs");
kvData.addMetadata("OWNER", "ozone"); kvData.addMetadata("OWNER", "ozone");
kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED); kvData.setState(ContainerProtos.ContainerDataProto.State.CLOSED);
ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
@ -117,7 +117,7 @@ public class TestContainerDataYaml {
assertEquals("RocksDB", kvData.getContainerDBType()); assertEquals("RocksDB", kvData.getContainerDBType());
assertEquals(containerFile.getParent(), kvData.getMetadataPath()); assertEquals(containerFile.getParent(), kvData.getMetadataPath());
assertEquals(containerFile.getParent(), kvData.getChunksPath()); assertEquals(containerFile.getParent(), kvData.getChunksPath());
assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED, kvData
.getState()); .getState());
assertEquals(1, kvData.getLayOutVersion()); assertEquals(1, kvData.getLayOutVersion());
assertEquals(2, kvData.getMetadata().size()); assertEquals(2, kvData.getMetadata().size());
@ -161,7 +161,7 @@ public class TestContainerDataYaml {
ContainerUtils.verifyChecksum(kvData); ContainerUtils.verifyChecksum(kvData);
//Checking the Container file data is consistent or not //Checking the Container file data is consistent or not
assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED, kvData
.getState()); .getState());
assertEquals("RocksDB", kvData.getContainerDBType()); assertEquals("RocksDB", kvData.getContainerDBType());
assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData

View File

@ -51,8 +51,8 @@ public class TestContainerSet {
public void testAddGetRemoveContainer() throws StorageContainerException { public void testAddGetRemoveContainer() throws StorageContainerException {
ContainerSet containerSet = new ContainerSet(); ContainerSet containerSet = new ContainerSet();
long containerId = 100L; long containerId = 100L;
ContainerProtos.ContainerLifeCycleState state = ContainerProtos ContainerProtos.ContainerDataProto.State state = ContainerProtos
.ContainerLifeCycleState.CLOSED; .ContainerDataProto.State.CLOSED;
KeyValueContainerData kvData = new KeyValueContainerData(containerId, KeyValueContainerData kvData = new KeyValueContainerData(containerId,
(long) StorageUnit.GB.toBytes(5)); (long) StorageUnit.GB.toBytes(5));
@ -101,10 +101,10 @@ public class TestContainerSet {
ContainerData containerData = kv.getContainerData(); ContainerData containerData = kv.getContainerData();
long containerId = containerData.getContainerID(); long containerId = containerData.getContainerID();
if (containerId%2 == 0) { if (containerId%2 == 0) {
assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
containerData.getState()); containerData.getState());
} else { } else {
assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, assertEquals(ContainerProtos.ContainerDataProto.State.OPEN,
containerData.getState()); containerData.getState());
} }
count++; count++;
@ -121,10 +121,10 @@ public class TestContainerSet {
ContainerData containerData = kv.getContainerData(); ContainerData containerData = kv.getContainerData();
long containerId = containerData.getContainerID(); long containerId = containerData.getContainerID();
if (containerId%2 == 0) { if (containerId%2 == 0) {
assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
containerData.getState()); containerData.getState());
} else { } else {
assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, assertEquals(ContainerProtos.ContainerDataProto.State.OPEN,
containerData.getState()); containerData.getState());
} }
count++; count++;
@ -168,9 +168,9 @@ public class TestContainerSet {
KeyValueContainerData kvData = new KeyValueContainerData(i, KeyValueContainerData kvData = new KeyValueContainerData(i,
(long) StorageUnit.GB.toBytes(5)); (long) StorageUnit.GB.toBytes(5));
if (i%2 == 0) { if (i%2 == 0) {
kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED); kvData.setState(ContainerProtos.ContainerDataProto.State.CLOSED);
} else { } else {
kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN); kvData.setState(ContainerProtos.ContainerDataProto.State.OPEN);
} }
KeyValueContainer kv = new KeyValueContainer(kvData, new KeyValueContainer kv = new KeyValueContainer(kvData, new
OzoneConfiguration()); OzoneConfiguration());

View File

@ -24,8 +24,6 @@ import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerLifeCycleState;
import org.apache.hadoop.hdds.scm.container.common.helpers import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException; .StorageContainerException;
import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.BlockData;
@ -185,7 +183,8 @@ public class TestKeyValueContainer {
keyValueContainerData = keyValueContainer keyValueContainerData = keyValueContainer
.getContainerData(); .getContainerData();
keyValueContainerData.setState(ContainerLifeCycleState.CLOSED); keyValueContainerData.setState(
ContainerProtos.ContainerDataProto.State.CLOSED);
int numberOfKeysToWrite = 12; int numberOfKeysToWrite = 12;
//write one few keys to check the key count after import //write one few keys to check the key count after import
@ -286,7 +285,7 @@ public class TestKeyValueContainer {
@Test @Test
public void testDeleteContainer() throws Exception { public void testDeleteContainer() throws Exception {
keyValueContainerData.setState(ContainerProtos.ContainerLifeCycleState keyValueContainerData.setState(ContainerProtos.ContainerDataProto.State
.CLOSED); .CLOSED);
keyValueContainer = new KeyValueContainer( keyValueContainer = new KeyValueContainer(
keyValueContainerData, conf); keyValueContainerData, conf);
@ -315,7 +314,7 @@ public class TestKeyValueContainer {
keyValueContainerData = keyValueContainer keyValueContainerData = keyValueContainer
.getContainerData(); .getContainerData();
assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
keyValueContainerData.getState()); keyValueContainerData.getState());
//Check state in the .container file //Check state in the .container file
@ -325,7 +324,7 @@ public class TestKeyValueContainer {
keyValueContainerData = (KeyValueContainerData) ContainerDataYaml keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
.readContainerFile(containerFile); .readContainerFile(containerFile);
assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
keyValueContainerData.getState()); keyValueContainerData.getState());
} }
@ -354,8 +353,8 @@ public class TestKeyValueContainer {
@Test @Test
public void testUpdateContainerUnsupportedRequest() throws Exception { public void testUpdateContainerUnsupportedRequest() throws Exception {
try { try {
keyValueContainerData.setState(ContainerProtos.ContainerLifeCycleState keyValueContainerData.setState(
.CLOSED); ContainerProtos.ContainerDataProto.State.CLOSED);
keyValueContainer = new KeyValueContainer(keyValueContainerData, conf); keyValueContainer = new KeyValueContainer(keyValueContainerData, conf);
keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId); keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
Map<String, String> metadata = new HashMap<>(); Map<String, String> metadata = new HashMap<>();

View File

@ -266,7 +266,7 @@ public class TestKeyValueHandler {
KeyValueContainerData kvData = new KeyValueContainerData(containerID, KeyValueContainerData kvData = new KeyValueContainerData(containerID,
(long) StorageUnit.GB.toBytes(1)); (long) StorageUnit.GB.toBytes(1));
KeyValueContainer container = new KeyValueContainer(kvData, conf); KeyValueContainer container = new KeyValueContainer(kvData, conf);
kvData.setState(ContainerProtos.ContainerLifeCycleState.INVALID); kvData.setState(ContainerProtos.ContainerDataProto.State.INVALID);
// Create Close container request // Create Close container request
ContainerCommandRequestProto closeContainerRequest = ContainerCommandRequestProto closeContainerRequest =

View File

@ -78,7 +78,7 @@ public class ContainerReportHandler implements
Set<ContainerID> containerIds = containerReport.getReportsList().stream() Set<ContainerID> containerIds = containerReport.getReportsList().stream()
.map(StorageContainerDatanodeProtocolProtos .map(StorageContainerDatanodeProtocolProtos
.ContainerInfo::getContainerID) .ContainerReplicaProto::getContainerID)
.map(ContainerID::new) .map(ContainerID::new)
.collect(Collectors.toSet()); .collect(Collectors.toSet());

View File

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

View File

@ -55,7 +55,7 @@ public final class HddsTestUtils {
*/ */
public static NodeRegistrationContainerReport public static NodeRegistrationContainerReport
createNodeRegistrationContainerReport(List<ContainerInfo> dnContainers) { createNodeRegistrationContainerReport(List<ContainerInfo> dnContainers) {
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> List<StorageContainerDatanodeProtocolProtos.ContainerReplicaProto>
containers = new ArrayList<>(); containers = new ArrayList<>();
dnContainers.forEach(c -> { dnContainers.forEach(c -> {
containers.add(TestUtils.getRandomContainerInfo(c.getContainerID())); containers.add(TestUtils.getRandomContainerInfo(c.getContainerID()));

View File

@ -17,6 +17,8 @@
package org.apache.hadoop.hdds.scm; package org.apache.hadoop.hdds.scm;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReport; .StorageContainerDatanodeProtocolProtos.PipelineReport;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -27,8 +29,6 @@ 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.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol import org.apache.hadoop.hdds.protocol
@ -293,7 +293,7 @@ public final class TestUtils {
*/ */
public static ContainerReportsProto getRandomContainerReports( public static ContainerReportsProto getRandomContainerReports(
int numberOfContainers) { int numberOfContainers) {
List<ContainerInfo> containerInfos = new ArrayList<>(); List<ContainerReplicaProto> containerInfos = new ArrayList<>();
for (int i = 0; i < numberOfContainers; i++) { for (int i = 0; i < numberOfContainers; i++) {
containerInfos.add(getRandomContainerInfo(i)); containerInfos.add(getRandomContainerInfo(i));
} }
@ -326,7 +326,7 @@ public final class TestUtils {
* @return ContainerReportsProto * @return ContainerReportsProto
*/ */
public static ContainerReportsProto getContainerReports( public static ContainerReportsProto getContainerReports(
ContainerInfo... containerInfos) { ContainerReplicaProto... containerInfos) {
return getContainerReports(Arrays.asList(containerInfos)); return getContainerReports(Arrays.asList(containerInfos));
} }
@ -338,10 +338,10 @@ public final class TestUtils {
* @return ContainerReportsProto * @return ContainerReportsProto
*/ */
public static ContainerReportsProto getContainerReports( public static ContainerReportsProto getContainerReports(
List<ContainerInfo> containerInfos) { List<ContainerReplicaProto> containerInfos) {
ContainerReportsProto.Builder ContainerReportsProto.Builder
reportsBuilder = ContainerReportsProto.newBuilder(); reportsBuilder = ContainerReportsProto.newBuilder();
for (ContainerInfo containerInfo : containerInfos) { for (ContainerReplicaProto containerInfo : containerInfos) {
reportsBuilder.addReports(containerInfo); reportsBuilder.addReports(containerInfo);
} }
return reportsBuilder.build(); return reportsBuilder.build();
@ -354,7 +354,8 @@ public final class TestUtils {
* *
* @return ContainerInfo * @return ContainerInfo
*/ */
public static ContainerInfo getRandomContainerInfo(long containerId) { public static ContainerReplicaProto getRandomContainerInfo(
long containerId) {
return createContainerInfo(containerId, return createContainerInfo(containerId,
OzoneConsts.GB * 5, OzoneConsts.GB * 5,
random.nextLong(1000), random.nextLong(1000),
@ -379,11 +380,12 @@ public final class TestUtils {
* *
* @return ContainerInfo * @return ContainerInfo
*/ */
public static ContainerInfo createContainerInfo( public static ContainerReplicaProto createContainerInfo(
long containerId, long size, long keyCount, long bytesUsed, long containerId, long size, long keyCount, long bytesUsed,
long readCount, long readBytes, long writeCount, long writeBytes) { long readCount, long readBytes, long writeCount, long writeBytes) {
return ContainerInfo.newBuilder() return ContainerReplicaProto.newBuilder()
.setContainerID(containerId) .setContainerID(containerId)
.setState(ContainerReplicaProto.State.OPEN)
.setSize(size) .setSize(size)
.setKeyCount(keyCount) .setKeyCount(keyCount)
.setUsed(bytesUsed) .setUsed(bytesUsed)

View File

@ -191,9 +191,11 @@ public class TestContainerReportHandler implements EventPublisher {
for (long containerId : containerIds) { for (long containerId : containerIds) {
org.apache.hadoop.hdds.protocol.proto org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder .StorageContainerDatanodeProtocolProtos
.ContainerReplicaProto.Builder
ciBuilder = org.apache.hadoop.hdds.protocol.proto ciBuilder = org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); .StorageContainerDatanodeProtocolProtos
.ContainerReplicaProto.newBuilder();
ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2") ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2")
.setSize(5368709120L) .setSize(5368709120L)
.setUsed(2000000000L) .setUsed(2000000000L)

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
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.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
@ -244,10 +244,10 @@ public class TestSCMContainerManager {
public void testFullContainerReport() throws Exception { public void testFullContainerReport() throws Exception {
ContainerInfo info = createContainer(); ContainerInfo info = createContainer();
DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails(); DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports = List<ContainerReplicaProto> reports =
new ArrayList<>(); new ArrayList<>();
StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = ContainerReplicaProto.Builder ciBuilder =
StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); ContainerReplicaProto.newBuilder();
ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2") ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2")
.setSize(5368709120L) .setSize(5368709120L)
.setUsed(2000000000L) .setUsed(2000000000L)
@ -257,6 +257,7 @@ public class TestSCMContainerManager {
.setReadBytes(2000000000L) .setReadBytes(2000000000L)
.setWriteBytes(2000000000L) .setWriteBytes(2000000000L)
.setContainerID(info.getContainerID()) .setContainerID(info.getContainerID())
.setState(ContainerReplicaProto.State.CLOSED)
.setDeleteTransactionId(0); .setDeleteTransactionId(0);
reports.add(ciBuilder.build()); reports.add(ciBuilder.build());
@ -274,14 +275,14 @@ public class TestSCMContainerManager {
updatedContainer.getNumberOfKeys()); updatedContainer.getNumberOfKeys());
Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes()); Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) { for (ContainerReplicaProto c : reports) {
Assert.assertEquals(containerManager.getContainerReplicas( Assert.assertEquals(containerManager.getContainerReplicas(
ContainerID.valueof(c.getContainerID())).size(), 1); ContainerID.valueof(c.getContainerID())).size(), 1);
} }
containerManager.processContainerReports(TestUtils.randomDatanodeDetails(), containerManager.processContainerReports(TestUtils.randomDatanodeDetails(),
crBuilder.build()); crBuilder.build());
for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) { for (ContainerReplicaProto c : reports) {
Assert.assertEquals(containerManager.getContainerReplicas( Assert.assertEquals(containerManager.getContainerReplicas(
ContainerID.valueof(c.getContainerID())).size(), 2); ContainerID.valueof(c.getContainerID())).size(), 2);
} }
@ -292,10 +293,10 @@ public class TestSCMContainerManager {
ContainerInfo info1 = createContainer(); ContainerInfo info1 = createContainer();
ContainerInfo info2 = createContainer(); ContainerInfo info2 = createContainer();
DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails(); DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports = List<ContainerReplicaProto> reports =
new ArrayList<>(); new ArrayList<>();
StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = ContainerReplicaProto.Builder ciBuilder =
StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); ContainerReplicaProto.newBuilder();
long cID1 = info1.getContainerID(); long cID1 = info1.getContainerID();
long cID2 = info2.getContainerID(); long cID2 = info2.getContainerID();
ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2") ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2")
@ -304,7 +305,8 @@ public class TestSCMContainerManager {
.setKeyCount(100000000L) .setKeyCount(100000000L)
.setReadBytes(1000000000L) .setReadBytes(1000000000L)
.setWriteBytes(1000000000L) .setWriteBytes(1000000000L)
.setContainerID(cID1); .setContainerID(cID1)
.setState(ContainerReplicaProto.State.CLOSED);
reports.add(ciBuilder.build()); reports.add(ciBuilder.build());
ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea54a9") ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea54a9")

View File

@ -23,9 +23,7 @@ import java.util.stream.Collectors;
import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.cli.HddsVersionProvider;
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
.ContainerData; .ContainerDataProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerLifeCycleState;
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 import org.apache.hadoop.hdds.scm.container.common.helpers
@ -64,13 +62,13 @@ public class InfoSubcommand implements Callable<Void> {
getContainerWithPipeline(containerID); getContainerWithPipeline(containerID);
Preconditions.checkNotNull(container, "Container cannot be null"); Preconditions.checkNotNull(container, "Container cannot be null");
ContainerData containerData = scmClient.readContainer(container ContainerDataProto containerData = scmClient.readContainer(container
.getContainerInfo().getContainerID(), container.getPipeline()); .getContainerInfo().getContainerID(), container.getPipeline());
// Print container report info. // Print container report info.
LOG.info("Container id: {}", containerID); LOG.info("Container id: {}", containerID);
String openStatus = String openStatus =
containerData.getState() == ContainerLifeCycleState.OPEN ? "OPEN" : containerData.getState() == ContainerDataProto.State.OPEN ? "OPEN" :
"CLOSED"; "CLOSED";
LOG.info("Container State: {}", openStatus); LOG.info("Container State: {}", openStatus);
LOG.info("Container Path: {}", containerData.getContainerPath()); LOG.info("Container Path: {}", containerData.getContainerPath());

View File

@ -464,7 +464,7 @@ public class TestCloseContainerHandlingByClient {
if (datanodes.get(0).equals(datanodeService.getDatanodeDetails())) { if (datanodes.get(0).equals(datanodeService.getDatanodeDetails())) {
datanodeService.getDatanodeStateMachine().getContainer() datanodeService.getDatanodeStateMachine().getContainer()
.getContainerSet().getContainer(containerID).getContainerData() .getContainerSet().getContainer(containerID).getContainerData()
.setState(ContainerProtos.ContainerLifeCycleState.CLOSING); .setState(ContainerProtos.ContainerDataProto.State.CLOSING);
} }
} }
dataString = fixedLengthString(keyString, (chunkSize * 1 / 2)); dataString = fixedLengthString(keyString, (chunkSize * 1 / 2));

View File

@ -161,7 +161,7 @@ public class TestContainerStateMachineFailures {
.getContainer().getContainerSet() .getContainer().getContainerSet()
.getContainer(omKeyLocationInfo.getContainerID()) .getContainer(omKeyLocationInfo.getContainerID())
.getContainerState() .getContainerState()
== ContainerProtos.ContainerLifeCycleState.UNHEALTHY); == ContainerProtos.ContainerDataProto.State.UNHEALTHY);
try { try {
// subsequent requests will fail with unhealthy container exception // subsequent requests will fail with unhealthy container exception
key.close(); key.close();

View File

@ -268,10 +268,10 @@ public class TestContainerPersistence {
// ContainerSet#getContainerReport currently returns all containers (open // ContainerSet#getContainerReport currently returns all containers (open
// and closed) reports. // and closed) reports.
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports = List<StorageContainerDatanodeProtocolProtos.ContainerReplicaProto> reports =
containerSet.getContainerReport().getReportsList(); containerSet.getContainerReport().getReportsList();
Assert.assertEquals(10, reports.size()); Assert.assertEquals(10, reports.size());
for (StorageContainerDatanodeProtocolProtos.ContainerInfo report : for (StorageContainerDatanodeProtocolProtos.ContainerReplicaProto report :
reports) { reports) {
long actualContainerID = report.getContainerID(); long actualContainerID = report.getContainerID();
Assert.assertTrue(containerIDs.remove(actualContainerID)); Assert.assertTrue(containerIDs.remove(actualContainerID));

View File

@ -24,7 +24,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.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.ContainerInfo; .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.block.DeletedBlockLogImpl; import org.apache.hadoop.hdds.scm.block.DeletedBlockLogImpl;
@ -233,9 +233,11 @@ public class TestBlockDeletion {
ContainerReportsProto containerReport = dnContainerSet.getContainerReport(); ContainerReportsProto containerReport = dnContainerSet.getContainerReport();
ContainerReportsProto.Builder dummyReportsBuilder = ContainerReportsProto.Builder dummyReportsBuilder =
ContainerReportsProto.newBuilder(); ContainerReportsProto.newBuilder();
for (ContainerInfo containerInfo : containerReport.getReportsList()) { for (ContainerReplicaProto containerInfo :
containerReport.getReportsList()) {
dummyReportsBuilder.addReports( dummyReportsBuilder.addReports(
ContainerInfo.newBuilder(containerInfo).setDeleteTransactionId(0) ContainerReplicaProto.newBuilder(containerInfo)
.setDeleteTransactionId(0)
.build()); .build());
} }
ContainerReportsProto dummyReport = dummyReportsBuilder.build(); ContainerReportsProto dummyReport = dummyReportsBuilder.build();
@ -246,7 +248,7 @@ public class TestBlockDeletion {
// 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();
for (ContainerInfo containerInfo : dummyReport.getReportsList()) { for (ContainerReplicaProto containerInfo : dummyReport.getReportsList()) {
long containerId = containerInfo.getContainerID(); long containerId = containerInfo.getContainerID();
// Event should be triggered only for containers which have deleted blocks // Event should be triggered only for containers which have deleted blocks
if (containerIdsWithDeletedBlocks.contains(containerId)) { if (containerIdsWithDeletedBlocks.contains(containerId)) {

View File

@ -469,7 +469,7 @@ public class SQLCLI extends Configured implements Tool {
long containerID = Longs.fromByteArray(key); long containerID = Longs.fromByteArray(key);
ContainerInfo containerInfo = null; ContainerInfo containerInfo = null;
containerInfo = ContainerInfo.fromProtobuf( containerInfo = ContainerInfo.fromProtobuf(
HddsProtos.SCMContainerInfo.PARSER.parseFrom(value)); HddsProtos.ContainerInfoProto.PARSER.parseFrom(value));
Preconditions.checkNotNull(containerInfo); Preconditions.checkNotNull(containerInfo);
try { try {
//TODO: include container state to sqllite schema //TODO: include container state to sqllite schema