HDDS-1016. Allow marking containers as unhealthy. Contributed by Arpit Agarwal.
This commit is contained in:
parent
7456fc99ee
commit
c35419579b
@ -84,6 +84,11 @@ void update(Map<String, String> metaData, boolean forceUpdate)
|
|||||||
*/
|
*/
|
||||||
void markContainerForClose() throws StorageContainerException;
|
void markContainerForClose() throws StorageContainerException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marks the container replica as unhealthy.
|
||||||
|
*/
|
||||||
|
void markContainerUnhealthy() throws StorageContainerException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Quasi Closes a open container, if it is already closed or does not exist a
|
* Quasi Closes a open container, if it is already closed or does not exist a
|
||||||
* StorageContainerException is thrown.
|
* StorageContainerException is thrown.
|
||||||
|
@ -64,6 +64,7 @@
|
|||||||
.Result.CONTAINER_FILES_CREATE_ERROR;
|
.Result.CONTAINER_FILES_CREATE_ERROR;
|
||||||
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
.Result.CONTAINER_INTERNAL_ERROR;
|
.Result.CONTAINER_INTERNAL_ERROR;
|
||||||
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_OPEN;
|
||||||
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
.Result.DISK_OUT_OF_SPACE;
|
.Result.DISK_OUT_OF_SPACE;
|
||||||
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
@ -72,6 +73,7 @@
|
|||||||
.Result.INVALID_CONTAINER_STATE;
|
.Result.INVALID_CONTAINER_STATE;
|
||||||
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
.Result.UNSUPPORTED_REQUEST;
|
.Result.UNSUPPORTED_REQUEST;
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
@ -109,8 +111,8 @@ public void create(VolumeSet volumeSet, VolumeChoosingPolicy
|
|||||||
|
|
||||||
File containerMetaDataPath = null;
|
File containerMetaDataPath = null;
|
||||||
//acquiring volumeset read lock
|
//acquiring volumeset read lock
|
||||||
volumeSet.readLock();
|
|
||||||
long maxSize = containerData.getMaxSize();
|
long maxSize = containerData.getMaxSize();
|
||||||
|
volumeSet.readLock();
|
||||||
try {
|
try {
|
||||||
HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
|
HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
|
||||||
.getVolumesList(), maxSize);
|
.getVolumesList(), maxSize);
|
||||||
@ -270,28 +272,67 @@ public void delete(boolean forceDelete)
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void markContainerForClose() throws StorageContainerException {
|
public void markContainerForClose() throws StorageContainerException {
|
||||||
|
writeLock();
|
||||||
|
try {
|
||||||
|
if (getContainerState() != ContainerDataProto.State.OPEN) {
|
||||||
|
throw new StorageContainerException(
|
||||||
|
"Attempting to close a " + getContainerState() + " container.",
|
||||||
|
CONTAINER_NOT_OPEN);
|
||||||
|
}
|
||||||
updateContainerData(() ->
|
updateContainerData(() ->
|
||||||
containerData.setState(ContainerDataProto.State.CLOSING));
|
containerData.setState(ContainerDataProto.State.CLOSING));
|
||||||
|
} finally {
|
||||||
|
writeUnlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void markContainerUnhealthy() throws StorageContainerException {
|
||||||
|
writeLock();
|
||||||
|
try {
|
||||||
|
updateContainerData(() ->
|
||||||
|
containerData.setState(ContainerDataProto.State.UNHEALTHY));
|
||||||
|
} finally {
|
||||||
|
writeUnlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void quasiClose() throws StorageContainerException {
|
public void quasiClose() throws StorageContainerException {
|
||||||
|
writeLock();
|
||||||
|
try {
|
||||||
updateContainerData(containerData::quasiCloseContainer);
|
updateContainerData(containerData::quasiCloseContainer);
|
||||||
|
} finally {
|
||||||
|
writeUnlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws StorageContainerException {
|
public void close() throws StorageContainerException {
|
||||||
|
writeLock();
|
||||||
|
try {
|
||||||
updateContainerData(containerData::closeContainer);
|
updateContainerData(containerData::closeContainer);
|
||||||
|
} finally {
|
||||||
|
writeUnlock();
|
||||||
|
}
|
||||||
|
|
||||||
// It is ok if this operation takes a bit of time.
|
// It is ok if this operation takes a bit of time.
|
||||||
// Close container is not expected to be instantaneous.
|
// Close container is not expected to be instantaneous.
|
||||||
compactDB();
|
compactDB();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Must be invoked with the writeLock held.
|
||||||
|
*
|
||||||
|
* @param update
|
||||||
|
* @throws StorageContainerException
|
||||||
|
*/
|
||||||
private void updateContainerData(Runnable update)
|
private void updateContainerData(Runnable update)
|
||||||
throws StorageContainerException {
|
throws StorageContainerException {
|
||||||
|
Preconditions.checkState(hasWriteLock());
|
||||||
ContainerDataProto.State oldState = null;
|
ContainerDataProto.State oldState = null;
|
||||||
try {
|
try {
|
||||||
writeLock();
|
|
||||||
oldState = containerData.getState();
|
oldState = containerData.getState();
|
||||||
update.run();
|
update.run();
|
||||||
File containerFile = getContainerFile();
|
File containerFile = getContainerFile();
|
||||||
@ -304,12 +345,10 @@ private void updateContainerData(Runnable update)
|
|||||||
containerData.setState(oldState);
|
containerData.setState(oldState);
|
||||||
}
|
}
|
||||||
throw ex;
|
throw ex;
|
||||||
} finally {
|
|
||||||
writeUnlock();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void compactDB() throws StorageContainerException {
|
void compactDB() throws StorageContainerException {
|
||||||
try {
|
try {
|
||||||
MetadataStore db = BlockUtils.getDB(containerData, config);
|
MetadataStore db = BlockUtils.getDB(containerData, config);
|
||||||
db.compactDB();
|
db.compactDB();
|
||||||
@ -340,7 +379,8 @@ public ContainerType getContainerType() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void update(Map<String, String> metadata, boolean forceUpdate)
|
public void update(
|
||||||
|
Map<String, String> metadata, boolean forceUpdate)
|
||||||
throws StorageContainerException {
|
throws StorageContainerException {
|
||||||
|
|
||||||
// TODO: Now, when writing the updated data to .container file, we are
|
// TODO: Now, when writing the updated data to .container file, we are
|
||||||
|
@ -111,7 +111,9 @@ public class KeyValueHandler extends Handler {
|
|||||||
private final BlockDeletingService blockDeletingService;
|
private final BlockDeletingService blockDeletingService;
|
||||||
private final VolumeChoosingPolicy volumeChoosingPolicy;
|
private final VolumeChoosingPolicy volumeChoosingPolicy;
|
||||||
private final long maxContainerSize;
|
private final long maxContainerSize;
|
||||||
private final AutoCloseableLock handlerLock;
|
|
||||||
|
// A lock that is held during container creation.
|
||||||
|
private final AutoCloseableLock containerCreationLock;
|
||||||
private final boolean doSyncWrite;
|
private final boolean doSyncWrite;
|
||||||
|
|
||||||
public KeyValueHandler(Configuration config, StateContext context,
|
public KeyValueHandler(Configuration config, StateContext context,
|
||||||
@ -143,7 +145,7 @@ public KeyValueHandler(Configuration config, StateContext context,
|
|||||||
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
|
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
|
||||||
// this handler lock is used for synchronizing createContainer Requests,
|
// this handler lock is used for synchronizing createContainer Requests,
|
||||||
// so using a fair lock here.
|
// so using a fair lock here.
|
||||||
handlerLock = new AutoCloseableLock(new ReentrantLock(true));
|
containerCreationLock = new AutoCloseableLock(new ReentrantLock(true));
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
@ -212,7 +214,7 @@ public BlockManager getBlockManager() {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Handles Create Container Request. If successful, adds the container to
|
* Handles Create Container Request. If successful, adds the container to
|
||||||
* ContainerSet.
|
* ContainerSet and sends an ICR to the SCM.
|
||||||
*/
|
*/
|
||||||
ContainerCommandResponseProto handleCreateContainer(
|
ContainerCommandResponseProto handleCreateContainer(
|
||||||
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
|
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
|
||||||
@ -235,14 +237,12 @@ ContainerCommandResponseProto handleCreateContainer(
|
|||||||
KeyValueContainer newContainer = new KeyValueContainer(
|
KeyValueContainer newContainer = new KeyValueContainer(
|
||||||
newContainerData, conf);
|
newContainerData, conf);
|
||||||
|
|
||||||
try {
|
boolean created = false;
|
||||||
handlerLock.acquire();
|
try (AutoCloseableLock l = containerCreationLock.acquire()) {
|
||||||
if (containerSet.getContainer(containerID) == null) {
|
if (containerSet.getContainer(containerID) == null) {
|
||||||
newContainer.create(volumeSet, volumeChoosingPolicy, scmID);
|
newContainer.create(volumeSet, volumeChoosingPolicy, scmID);
|
||||||
containerSet.addContainer(newContainer);
|
created = containerSet.addContainer(newContainer);
|
||||||
sendICR(newContainer);
|
|
||||||
} else {
|
} else {
|
||||||
|
|
||||||
// The create container request for an already existing container can
|
// The create container request for an already existing container can
|
||||||
// arrive in case the ContainerStateMachine reapplies the transaction
|
// arrive in case the ContainerStateMachine reapplies the transaction
|
||||||
// on datanode restart. Just log a warning msg here.
|
// on datanode restart. Just log a warning msg here.
|
||||||
@ -251,10 +251,15 @@ ContainerCommandResponseProto handleCreateContainer(
|
|||||||
}
|
}
|
||||||
} catch (StorageContainerException ex) {
|
} catch (StorageContainerException ex) {
|
||||||
return ContainerUtils.logAndReturnError(LOG, ex, request);
|
return ContainerUtils.logAndReturnError(LOG, ex, request);
|
||||||
} finally {
|
|
||||||
handlerLock.release();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (created) {
|
||||||
|
try {
|
||||||
|
sendICR(newContainer);
|
||||||
|
} catch (StorageContainerException ex) {
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, ex, request);
|
||||||
|
}
|
||||||
|
}
|
||||||
return ContainerUtils.getSuccessResponse(request);
|
return ContainerUtils.getSuccessResponse(request);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -282,6 +287,14 @@ ContainerCommandResponseProto handleReadContainer(
|
|||||||
return ContainerUtils.malformedRequest(request);
|
return ContainerUtils.malformedRequest(request);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// The container can become unhealthy after the lock is released.
|
||||||
|
// The operation will likely fail/timeout in that happens.
|
||||||
|
try {
|
||||||
|
checkContainerIsHealthy(kvContainer);
|
||||||
|
} catch (StorageContainerException sce) {
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, sce, request);
|
||||||
|
}
|
||||||
|
|
||||||
KeyValueContainerData containerData = kvContainer.getContainerData();
|
KeyValueContainerData containerData = kvContainer.getContainerData();
|
||||||
return KeyValueContainerUtil.getReadContainerResponse(
|
return KeyValueContainerUtil.getReadContainerResponse(
|
||||||
request, containerData);
|
request, containerData);
|
||||||
@ -420,6 +433,14 @@ ContainerCommandResponseProto handleGetBlock(
|
|||||||
return ContainerUtils.malformedRequest(request);
|
return ContainerUtils.malformedRequest(request);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// The container can become unhealthy after the lock is released.
|
||||||
|
// The operation will likely fail/timeout in that happens.
|
||||||
|
try {
|
||||||
|
checkContainerIsHealthy(kvContainer);
|
||||||
|
} catch (StorageContainerException sce) {
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, sce, request);
|
||||||
|
}
|
||||||
|
|
||||||
BlockData responseData;
|
BlockData responseData;
|
||||||
try {
|
try {
|
||||||
BlockID blockID = BlockID.getFromProtobuf(
|
BlockID blockID = BlockID.getFromProtobuf(
|
||||||
@ -451,6 +472,14 @@ ContainerCommandResponseProto handleGetCommittedBlockLength(
|
|||||||
return ContainerUtils.malformedRequest(request);
|
return ContainerUtils.malformedRequest(request);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// The container can become unhealthy after the lock is released.
|
||||||
|
// The operation will likely fail/timeout in that happens.
|
||||||
|
try {
|
||||||
|
checkContainerIsHealthy(kvContainer);
|
||||||
|
} catch (StorageContainerException sce) {
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, sce, request);
|
||||||
|
}
|
||||||
|
|
||||||
long blockLength;
|
long blockLength;
|
||||||
try {
|
try {
|
||||||
BlockID blockID = BlockID
|
BlockID blockID = BlockID
|
||||||
@ -510,6 +539,14 @@ ContainerCommandResponseProto handleReadChunk(
|
|||||||
return ContainerUtils.malformedRequest(request);
|
return ContainerUtils.malformedRequest(request);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// The container can become unhealthy after the lock is released.
|
||||||
|
// The operation will likely fail/timeout in that happens.
|
||||||
|
try {
|
||||||
|
checkContainerIsHealthy(kvContainer);
|
||||||
|
} catch (StorageContainerException sce) {
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, sce, request);
|
||||||
|
}
|
||||||
|
|
||||||
ChunkInfo chunkInfo;
|
ChunkInfo chunkInfo;
|
||||||
byte[] data;
|
byte[] data;
|
||||||
try {
|
try {
|
||||||
@ -537,6 +574,27 @@ ContainerCommandResponseProto handleReadChunk(
|
|||||||
return ChunkUtils.getReadChunkResponse(request, data, chunkInfo);
|
return ChunkUtils.getReadChunkResponse(request, data, chunkInfo);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Throw an exception if the container is unhealthy.
|
||||||
|
*
|
||||||
|
* @throws StorageContainerException if the container is unhealthy.
|
||||||
|
* @param kvContainer
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
void checkContainerIsHealthy(KeyValueContainer kvContainer)
|
||||||
|
throws StorageContainerException {
|
||||||
|
kvContainer.readLock();
|
||||||
|
try {
|
||||||
|
if (kvContainer.getContainerData().getState() == State.UNHEALTHY) {
|
||||||
|
throw new StorageContainerException(
|
||||||
|
"The container replica is unhealthy.",
|
||||||
|
CONTAINER_UNHEALTHY);
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
kvContainer.readUnlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handle Delete Chunk operation. Calls ChunkManager to process the request.
|
* Handle Delete Chunk operation. Calls ChunkManager to process the request.
|
||||||
*/
|
*/
|
||||||
@ -549,6 +607,14 @@ ContainerCommandResponseProto handleDeleteChunk(
|
|||||||
return ContainerUtils.malformedRequest(request);
|
return ContainerUtils.malformedRequest(request);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// The container can become unhealthy after the lock is released.
|
||||||
|
// The operation will likely fail/timeout in that happens.
|
||||||
|
try {
|
||||||
|
checkContainerIsHealthy(kvContainer);
|
||||||
|
} catch (StorageContainerException sce) {
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, sce, request);
|
||||||
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
checkContainerOpen(kvContainer);
|
checkContainerOpen(kvContainer);
|
||||||
|
|
||||||
@ -697,6 +763,14 @@ ContainerCommandResponseProto handleGetSmallFile(
|
|||||||
return ContainerUtils.malformedRequest(request);
|
return ContainerUtils.malformedRequest(request);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// The container can become unhealthy after the lock is released.
|
||||||
|
// The operation will likely fail/timeout in that happens.
|
||||||
|
try {
|
||||||
|
checkContainerIsHealthy(kvContainer);
|
||||||
|
} catch (StorageContainerException sce) {
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, sce, request);
|
||||||
|
}
|
||||||
|
|
||||||
GetSmallFileRequestProto getSmallFileReq = request.getGetSmallFile();
|
GetSmallFileRequestProto getSmallFileReq = request.getGetSmallFile();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
@ -0,0 +1,172 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.ozone.container.keyvalue;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.StorageUnit;
|
||||||
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||||
|
import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
|
||||||
|
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
|
||||||
|
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
|
||||||
|
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
import org.junit.rules.Timeout;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State.OPEN;
|
||||||
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State.UNHEALTHY;
|
||||||
|
import static org.hamcrest.core.Is.is;
|
||||||
|
import static org.junit.Assert.assertThat;
|
||||||
|
import static org.mockito.ArgumentMatchers.anyList;
|
||||||
|
import static org.mockito.ArgumentMatchers.anyLong;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests unhealthy container functionality in the {@link KeyValueContainer}
|
||||||
|
* class.
|
||||||
|
*/
|
||||||
|
public class TestKeyValueContainerMarkUnhealthy {
|
||||||
|
public static final Logger LOG = LoggerFactory.getLogger(
|
||||||
|
TestKeyValueContainerMarkUnhealthy.class);
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public TemporaryFolder folder = new TemporaryFolder();
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public Timeout timeout = new Timeout(600_000);
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
private OzoneConfiguration conf;
|
||||||
|
private String scmId = UUID.randomUUID().toString();
|
||||||
|
private VolumeSet volumeSet;
|
||||||
|
private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
|
||||||
|
private KeyValueContainerData keyValueContainerData;
|
||||||
|
private KeyValueContainer keyValueContainer;
|
||||||
|
private UUID datanodeId;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
conf = new OzoneConfiguration();
|
||||||
|
datanodeId = UUID.randomUUID();
|
||||||
|
HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
|
||||||
|
.getAbsolutePath()).conf(conf).datanodeUuid(datanodeId
|
||||||
|
.toString()).build();
|
||||||
|
|
||||||
|
volumeSet = mock(VolumeSet.class);
|
||||||
|
volumeChoosingPolicy = mock(RoundRobinVolumeChoosingPolicy.class);
|
||||||
|
Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
|
||||||
|
.thenReturn(hddsVolume);
|
||||||
|
|
||||||
|
keyValueContainerData = new KeyValueContainerData(1L,
|
||||||
|
(long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(),
|
||||||
|
datanodeId.toString());
|
||||||
|
final File metaDir = GenericTestUtils.getRandomizedTestDir();
|
||||||
|
metaDir.mkdirs();
|
||||||
|
keyValueContainerData.setMetadataPath(metaDir.getPath());
|
||||||
|
|
||||||
|
|
||||||
|
keyValueContainer = new KeyValueContainer(
|
||||||
|
keyValueContainerData, conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void teardown() {
|
||||||
|
volumeSet = null;
|
||||||
|
keyValueContainer = null;
|
||||||
|
keyValueContainerData = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verify that the .container file is correctly updated when a
|
||||||
|
* container is marked as unhealthy.
|
||||||
|
*
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testMarkContainerUnhealthy() throws IOException {
|
||||||
|
assertThat(keyValueContainerData.getState(), is(OPEN));
|
||||||
|
keyValueContainer.markContainerUnhealthy();
|
||||||
|
assertThat(keyValueContainerData.getState(), is(UNHEALTHY));
|
||||||
|
|
||||||
|
// Check metadata in the .container file
|
||||||
|
File containerFile = keyValueContainer.getContainerFile();
|
||||||
|
|
||||||
|
keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
|
||||||
|
.readContainerFile(containerFile);
|
||||||
|
assertThat(keyValueContainerData.getState(), is(UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Attempting to close an unhealthy container should fail.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testCloseUnhealthyContainer() throws IOException {
|
||||||
|
keyValueContainer.markContainerUnhealthy();
|
||||||
|
thrown.expect(StorageContainerException.class);
|
||||||
|
keyValueContainer.markContainerForClose();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Attempting to mark a closed container as unhealthy should succeed.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testMarkClosedContainerAsUnhealthy() throws IOException {
|
||||||
|
// We need to create the container so the compact-on-close operation
|
||||||
|
// does not NPE.
|
||||||
|
keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
|
||||||
|
keyValueContainer.close();
|
||||||
|
keyValueContainer.markContainerUnhealthy();
|
||||||
|
assertThat(keyValueContainerData.getState(), is(UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Attempting to mark a quasi-closed container as unhealthy should succeed.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testMarkQuasiClosedContainerAsUnhealthy() throws IOException {
|
||||||
|
keyValueContainer.quasiClose();
|
||||||
|
keyValueContainer.markContainerUnhealthy();
|
||||||
|
assertThat(keyValueContainerData.getState(), is(UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Attempting to mark a closing container as unhealthy should succeed.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testMarkClosingContainerAsUnhealthy() throws IOException {
|
||||||
|
keyValueContainer.markContainerForClose();
|
||||||
|
keyValueContainer.markContainerUnhealthy();
|
||||||
|
assertThat(keyValueContainerData.getState(), is(UNHEALTHY));
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,227 @@
|
|||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.ozone.container.keyvalue;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
|
||||||
|
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
||||||
|
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
|
||||||
|
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
|
||||||
|
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_UNHEALTHY;
|
||||||
|
import static org.hamcrest.core.Is.is;
|
||||||
|
import static org.junit.Assert.assertThat;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that KeyValueHandler fails certain operations when the
|
||||||
|
* container is unhealthy.
|
||||||
|
*/
|
||||||
|
public class TestKeyValueHandlerWithUnhealthyContainer {
|
||||||
|
public static final Logger LOG = LoggerFactory.getLogger(
|
||||||
|
TestKeyValueHandlerWithUnhealthyContainer.class);
|
||||||
|
|
||||||
|
private final static String DATANODE_UUID = UUID.randomUUID().toString();
|
||||||
|
private static final long DUMMY_CONTAINER_ID = 9999;
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRead() throws IOException {
|
||||||
|
KeyValueContainer container = getMockUnhealthyContainer();
|
||||||
|
KeyValueHandler handler = getDummyHandler();
|
||||||
|
|
||||||
|
ContainerProtos.ContainerCommandResponseProto response =
|
||||||
|
handler.handleReadContainer(
|
||||||
|
getDummyCommandRequestProto(ContainerProtos.Type.ReadContainer),
|
||||||
|
container);
|
||||||
|
assertThat(response.getResult(), is(CONTAINER_UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetBlock() throws IOException {
|
||||||
|
KeyValueContainer container = getMockUnhealthyContainer();
|
||||||
|
KeyValueHandler handler = getDummyHandler();
|
||||||
|
|
||||||
|
ContainerProtos.ContainerCommandResponseProto response =
|
||||||
|
handler.handleGetBlock(
|
||||||
|
getDummyCommandRequestProto(ContainerProtos.Type.GetBlock),
|
||||||
|
container);
|
||||||
|
assertThat(response.getResult(), is(CONTAINER_UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetCommittedBlockLength() throws IOException {
|
||||||
|
KeyValueContainer container = getMockUnhealthyContainer();
|
||||||
|
KeyValueHandler handler = getDummyHandler();
|
||||||
|
|
||||||
|
ContainerProtos.ContainerCommandResponseProto response =
|
||||||
|
handler.handleGetCommittedBlockLength(
|
||||||
|
getDummyCommandRequestProto(
|
||||||
|
ContainerProtos.Type.GetCommittedBlockLength),
|
||||||
|
container);
|
||||||
|
assertThat(response.getResult(), is(CONTAINER_UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReadChunk() throws IOException {
|
||||||
|
KeyValueContainer container = getMockUnhealthyContainer();
|
||||||
|
KeyValueHandler handler = getDummyHandler();
|
||||||
|
|
||||||
|
ContainerProtos.ContainerCommandResponseProto response =
|
||||||
|
handler.handleReadChunk(
|
||||||
|
getDummyCommandRequestProto(
|
||||||
|
ContainerProtos.Type.ReadChunk),
|
||||||
|
container, null);
|
||||||
|
assertThat(response.getResult(), is(CONTAINER_UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDeleteChunk() throws IOException {
|
||||||
|
KeyValueContainer container = getMockUnhealthyContainer();
|
||||||
|
KeyValueHandler handler = getDummyHandler();
|
||||||
|
|
||||||
|
ContainerProtos.ContainerCommandResponseProto response =
|
||||||
|
handler.handleDeleteChunk(
|
||||||
|
getDummyCommandRequestProto(
|
||||||
|
ContainerProtos.Type.DeleteChunk),
|
||||||
|
container);
|
||||||
|
assertThat(response.getResult(), is(CONTAINER_UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetSmallFile() throws IOException {
|
||||||
|
KeyValueContainer container = getMockUnhealthyContainer();
|
||||||
|
KeyValueHandler handler = getDummyHandler();
|
||||||
|
|
||||||
|
ContainerProtos.ContainerCommandResponseProto response =
|
||||||
|
handler.handleGetSmallFile(
|
||||||
|
getDummyCommandRequestProto(
|
||||||
|
ContainerProtos.Type.GetSmallFile),
|
||||||
|
container);
|
||||||
|
assertThat(response.getResult(), is(CONTAINER_UNHEALTHY));
|
||||||
|
}
|
||||||
|
|
||||||
|
// -- Helper methods below.
|
||||||
|
|
||||||
|
private KeyValueHandler getDummyHandler() throws IOException {
|
||||||
|
OzoneConfiguration conf = new OzoneConfiguration();
|
||||||
|
DatanodeDetails dnDetails = DatanodeDetails.newBuilder()
|
||||||
|
.setUuid(DATANODE_UUID)
|
||||||
|
.setHostName("dummyHost")
|
||||||
|
.setIpAddress("1.2.3.4")
|
||||||
|
.build();
|
||||||
|
DatanodeStateMachine stateMachine = mock(DatanodeStateMachine.class);
|
||||||
|
when(stateMachine.getDatanodeDetails()).thenReturn(dnDetails);
|
||||||
|
|
||||||
|
StateContext context = new StateContext(
|
||||||
|
conf, DatanodeStateMachine.DatanodeStates.RUNNING,
|
||||||
|
stateMachine);
|
||||||
|
|
||||||
|
return new KeyValueHandler(
|
||||||
|
new OzoneConfiguration(),
|
||||||
|
context,
|
||||||
|
mock(ContainerSet.class),
|
||||||
|
mock(VolumeSet.class),
|
||||||
|
mock(ContainerMetrics.class));
|
||||||
|
}
|
||||||
|
|
||||||
|
private KeyValueContainer getMockUnhealthyContainer() {
|
||||||
|
KeyValueContainerData containerData = mock(KeyValueContainerData.class);
|
||||||
|
when(containerData.getState()).thenReturn(
|
||||||
|
ContainerProtos.ContainerDataProto.State.UNHEALTHY);
|
||||||
|
return new KeyValueContainer(containerData, new OzoneConfiguration());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Construct fake protobuf messages for various types of requests.
|
||||||
|
* This is tedious, however necessary to test. Protobuf classes are final
|
||||||
|
* and cannot be mocked by Mockito.
|
||||||
|
*
|
||||||
|
* @param cmdType type of the container command.
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
private ContainerProtos.ContainerCommandRequestProto getDummyCommandRequestProto(
|
||||||
|
ContainerProtos.Type cmdType) {
|
||||||
|
final ContainerProtos.ContainerCommandRequestProto.Builder builder =
|
||||||
|
ContainerProtos.ContainerCommandRequestProto.newBuilder()
|
||||||
|
.setCmdType(cmdType)
|
||||||
|
.setContainerID(DUMMY_CONTAINER_ID)
|
||||||
|
.setDatanodeUuid(DATANODE_UUID);
|
||||||
|
|
||||||
|
final ContainerProtos.DatanodeBlockID fakeBlockId =
|
||||||
|
ContainerProtos.DatanodeBlockID.newBuilder()
|
||||||
|
.setContainerID(DUMMY_CONTAINER_ID).setLocalID(1).build();
|
||||||
|
|
||||||
|
final ContainerProtos.ChunkInfo fakeChunkInfo =
|
||||||
|
ContainerProtos.ChunkInfo.newBuilder()
|
||||||
|
.setChunkName("dummy")
|
||||||
|
.setOffset(0)
|
||||||
|
.setLen(100)
|
||||||
|
.setChecksumData(ContainerProtos.ChecksumData.newBuilder()
|
||||||
|
.setBytesPerChecksum(1)
|
||||||
|
.setType(ContainerProtos.ChecksumType.CRC32)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
|
||||||
|
switch(cmdType) {
|
||||||
|
case ReadContainer:
|
||||||
|
builder.setReadContainer(ContainerProtos.ReadContainerRequestProto.newBuilder().build());
|
||||||
|
break;
|
||||||
|
case GetBlock:
|
||||||
|
builder.setGetBlock(ContainerProtos.GetBlockRequestProto.newBuilder()
|
||||||
|
.setBlockID(fakeBlockId).build());
|
||||||
|
break;
|
||||||
|
case GetCommittedBlockLength:
|
||||||
|
builder.setGetCommittedBlockLength(
|
||||||
|
ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder()
|
||||||
|
.setBlockID(fakeBlockId).build());
|
||||||
|
case ReadChunk:
|
||||||
|
builder.setReadChunk(ContainerProtos.ReadChunkRequestProto.newBuilder()
|
||||||
|
.setBlockID(fakeBlockId).setChunkData(fakeChunkInfo).build());
|
||||||
|
break;
|
||||||
|
case DeleteChunk:
|
||||||
|
builder.setDeleteChunk(ContainerProtos.DeleteChunkRequestProto.newBuilder()
|
||||||
|
.setBlockID(fakeBlockId).setChunkData(fakeChunkInfo).build());
|
||||||
|
break;
|
||||||
|
case GetSmallFile:
|
||||||
|
builder.setGetSmallFile(ContainerProtos.GetSmallFileRequestProto.newBuilder()
|
||||||
|
.setBlock(ContainerProtos.GetBlockRequestProto.newBuilder()
|
||||||
|
.setBlockID(fakeBlockId)
|
||||||
|
.build())
|
||||||
|
.build());
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
Assert.fail("Unhandled request type " + cmdType + " in unit test");
|
||||||
|
}
|
||||||
|
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue
Block a user